Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-11-01 Thread Xinyu Liu

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review154420
---


Ship it!




THe RB looks great! I posted a couple of comments for further discussion, not 
to block the progress.


samza-operator/src/main/java/org/apache/samza/operators/api/MessageStreams.java 
(line 45)


This might be discussed before: why we use systemstreampartition instead of 
systemstream as the source? I think having an abstraction layer of stream will 
be much more convenient to the users to user this api. For example, instead of 

sources.foreach(source => source.map(...).filter()...)// per ssp handling

The api can be:
SystemMessageStream source = souces.get("system x", "stream y");
source.map(...).filter()...

This might not be a big deal if we only got one input stream. Things could 
get messy if we have more. For example, if we want to join, it will be much 
convenient to specify:

SystemMessageStream source1 = souces.get("system x", "stream 1");
SystemMessageStream source2 = souces.get("system x", "stream 2");
source1.join(source2, joinFunc)

Otherwise the user needs to join each partion and repeat the same logic for 
each partition. I couldn't think a case that a ssp level api will be useful 
since most the current stream processors operates on streams, not partitions.



samza-operator/src/main/java/org/apache/samza/operators/api/internal/Trigger.java
 (line 33)


The defination of early and late triggers seems to be a little different 
from what google dataflow has today. In Tyler's blog (stream 102), early and 
late are relevant if there is a watermark, which indicates the completion of 
input stream regarding to an event time. Early triggers happen before the 
watermark to solve the watermark being too slow problem and late triggers 
happen after the watermark to solve the watermark being too fast problem (late 
arrivals). For us, since there is no notion of watermark so far, it seems the 
semantics of early/late triggers are not well defined. I cannot tell when the 
window is closed (early or late tigger?) or can late trigger happens before 
early trigger? Seems the early and late meanings are loosely defined without a 
reference. Happy to talk about this more.


- Xinyu Liu


On Oct. 4, 2016, 12:45 a.m., Yi Pan (Data Infrastructure) wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/47835/
> ---
> 
> (Updated Oct. 4, 2016, 12:45 a.m.)
> 
> 
> Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
> Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.
> 
> 
> Bugs: SAMZA-914
> https://issues.apache.org/jira/browse/SAMZA-914
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> SAMZA-914: initial draft of operator programming API. Design doc attached to 
> SAMZA-914: 
> https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf
> 
> 
> Diffs
> -
> 
>   build.gradle 16facbbf4dff378c561461786ff186bd9eed 
>   gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
>   samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
> 5d066c5867e9df9e94e60bde825dedf10703b399 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStreams.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/TriggerBuilder.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/WindowState.java 
> PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/Message.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Operators.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Trigger.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/WindowFn.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/WindowOutput.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/ChainedOperators.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/ProcessorContext.java
>  P

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-10-04 Thread Yi Pan (Data Infrastructure)


> On Oct. 4, 2016, 4:34 p.m., Jagadish Venkatraman wrote:
> > samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java, 
> > line 40
> > 
> >
> > Why should this be public? Can't it be package private?

Yeah, I was originally thinking of Json ObjectMapper that would require the 
member variables to be public in deserialization. But there is no example here 
that uses ObjectMapper to read the bytes and construct this object. I will 
remove them for now.


- Yi


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review151345
---


On Oct. 4, 2016, 12:45 a.m., Yi Pan (Data Infrastructure) wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/47835/
> ---
> 
> (Updated Oct. 4, 2016, 12:45 a.m.)
> 
> 
> Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
> Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.
> 
> 
> Bugs: SAMZA-914
> https://issues.apache.org/jira/browse/SAMZA-914
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> SAMZA-914: initial draft of operator programming API. Design doc attached to 
> SAMZA-914: 
> https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf
> 
> 
> Diffs
> -
> 
>   build.gradle 16facbbf4dff378c561461786ff186bd9eed 
>   gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
>   samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
> 5d066c5867e9df9e94e60bde825dedf10703b399 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStreams.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/TriggerBuilder.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/WindowState.java 
> PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/Message.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Operators.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Trigger.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/WindowFn.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/WindowOutput.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/ChainedOperators.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/ProcessorContext.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/task/StreamOperatorAdaptorTask.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/TestTriggerBuilder.java
>  PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/TestWindows.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/data/TestIncomingSystemMessage.java
>  PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/data/TestLongOffset.java
>  PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/internal/TestOperators.java
>  PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/internal/TestTrigger.java
>  PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/impl/TestOutputMessage.java
>  PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/impl/TestProcessorContext.java
>  PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/impl/TestSimpleOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/impl/TestSinkOperatorImpl.java
>

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-10-04 Thread Jagadish Venkatraman

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review151345
---


Fix it, then Ship it!





samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java (line 
39)


Why should this be public? Can't it be package private?



samza-operator/src/test/java/org/apache/samza/task/WindowOperatorTask.java 
(line 38)


Why should this be `public`? Can't it be package ptivate?


- Jagadish Venkatraman


On Oct. 4, 2016, 12:45 a.m., Yi Pan (Data Infrastructure) wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/47835/
> ---
> 
> (Updated Oct. 4, 2016, 12:45 a.m.)
> 
> 
> Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
> Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.
> 
> 
> Bugs: SAMZA-914
> https://issues.apache.org/jira/browse/SAMZA-914
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> SAMZA-914: initial draft of operator programming API. Design doc attached to 
> SAMZA-914: 
> https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf
> 
> 
> Diffs
> -
> 
>   build.gradle 16facbbf4dff378c561461786ff186bd9eed 
>   gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
>   samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
> 5d066c5867e9df9e94e60bde825dedf10703b399 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStreams.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/TriggerBuilder.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/WindowState.java 
> PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/Message.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Operators.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Trigger.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/WindowFn.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/WindowOutput.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/ChainedOperators.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/ProcessorContext.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/task/StreamOperatorAdaptorTask.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/TestTriggerBuilder.java
>  PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/TestWindows.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/data/TestIncomingSystemMessage.java
>  PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/data/TestLongOffset.java
>  PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/internal/TestOperators.java
>  PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/internal/TestTrigger.java
>  PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/impl/TestOutputMessage.java
>  PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/impl/TestProcessorContext.java
>  PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/impl/TestSimpleOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/impl/TestSinkOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/impl/TestStateStoreImpl.java
>  

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-10-03 Thread Yi Pan (Data Infrastructure)

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/
---

(Updated Oct. 4, 2016, 12:45 a.m.)


Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.


Changes
---

Updated based on Jagadish and Jake's comments. Thanks, guys!


Bugs: SAMZA-914
https://issues.apache.org/jira/browse/SAMZA-914


Repository: samza


Description
---

SAMZA-914: initial draft of operator programming API. Design doc attached to 
SAMZA-914: 
https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf


Diffs (updated)
-

  build.gradle 16facbbf4dff378c561461786ff186bd9eed 
  gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
  samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
5d066c5867e9df9e94e60bde825dedf10703b399 
  
samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/MessageStreams.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/TriggerBuilder.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/WindowState.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/data/Message.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/internal/Operators.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/internal/Trigger.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/internal/WindowFn.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/internal/WindowOutput.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/ChainedOperators.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/ProcessorContext.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/task/StreamOperatorAdaptorTask.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java 
PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/api/TestTriggerBuilder.java
 PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/operators/api/TestWindows.java 
PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/api/data/TestIncomingSystemMessage.java
 PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/api/data/TestLongOffset.java
 PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/api/internal/TestOperators.java
 PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/api/internal/TestTrigger.java
 PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java
 PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/impl/TestOutputMessage.java
 PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/impl/TestProcessorContext.java
 PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/impl/TestSimpleOperatorImpl.java
 PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/impl/TestSinkOperatorImpl.java
 PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/impl/TestStateStoreImpl.java
 PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/impl/window/TestSessionWindowImpl.java
 PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/BroadcastOperatorTask.java 
PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/task/InputJsonSystemMessage.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java 
PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorAdaptorTask.java
 PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorTasks.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/WindowOperatorTask.java 
PRE-CREATION 
  
samza-sql-calcite/src/test/java/org/apache/samza/sql/calcite/schema/TestAvroSchemaConverter.java
 PRE-CREATION 
  samza-sql-core/README.md PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-10-03 Thread Yi Pan (Data Infrastructure)


> On Oct. 4, 2016, 12:10 a.m., Jake Maes wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java,
> >  line 505
> > 
> >
> > Where is this used? I couldn't find it on any of the 3 pages of the 
> > review.

I realized one use case of it during the review: consider ACG re-partition job 
that consumes 100+ topic partitions but all send to the same output. This is a 
typical merge case. Does it make sense?


> On Oct. 4, 2016, 12:10 a.m., Jake Maes wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/TriggerBuilder.java,
> >  line 65
> > 
> >
> > nit: I'd replace "early" with either "primary" or "regular". 
> > 
> > Early trigger contrasts late trigger with opposing terminology, but in 
> > terms of semantics, we really have a primary trigger, which is expected to 
> > cover the majority of the messages and then the late trigger to handle late 
> > arrivals. In that context, "early" doesn't make much sense because it 
> > doesn't sound like the normal case. 
> > 
> > If that^ understanding is correct, I'd suggest a rename.

The term is borrowed from Dataflow. It is better to stay w/ the same name w/ 
the origin, in my opinion, if we adopt the concept from the origin.


> On Oct. 4, 2016, 12:10 a.m., Jake Maes wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/TriggerBuilder.java,
> >  line 139
> > 
> >
> > This is essentially the same as "addTimeoutSinceFirstMessage" with a 
> > custom event time function, right?
> > 
> > Any other differences that I'm not seeing?
> > 
> > No action suggested, just making sure I understand.

Yes.


> On Oct. 4, 2016, 12:10 a.m., Jake Maes wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/TriggerBuilder.java,
> >  line 213
> > 
> >
> > Surprised to see these default implementations using system time rather 
> > than event time. Is it just because it's easier to ensure that system time 
> > exists and is valid?

This is not default implementation of "event time". This is specifically for 
system timeout trigger. Any event time based trigger is in 
earlyTriggerOnEventTime()


> On Oct. 4, 2016, 12:10 a.m., Jake Maes wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/TriggerBuilder.java,
> >  line 223
> > 
> >
> > Why would one put a size limit in a late trigger rather than an early 
> > trigger?

You don't want the late trigger to occur for each and every late arrivals 
either. This provides a way to suppress the late triggers.


> On Oct. 4, 2016, 12:10 a.m., Jake Maes wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/WindowState.java,
> >  line 47
> > 
> >
> > Why the terminology change? Here it's "earliest" and above it's "first"

Because this is talking about two very different characteristics: first 
indicate arrival order, while earliest is much more explicit regarding to 
temporal order in event time.


> On Oct. 4, 2016, 12:10 a.m., Jake Maes wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java, 
> > line 38
> > 
> >
> > Add a javadoc recommending a reboot if this class fails. 
> > 
> > Also, where's the "Start" button?
> > 
> > :-)

LoL.


> On Oct. 4, 2016, 12:10 a.m., Jake Maes wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java, 
> > line 95
> > 
> >
> > What's the advantage of building the trigger here rather than before 
> > invoking setTriggers()?

This is to hide the Trigger class from the user API.


> On Oct. 4, 2016, 12:10 a.m., Jake Maes wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/internal/WindowOutput.java,
> >  line 30
> > 
> >
> > This doesn't seem to add anything to Message. Is it just a placeholder 
> > in case we want to add something to window outputs and not messages? (For 
> > example, perhaps information about the trigger that fired.)
> > 
> > Is it the only implementation of Message?

Yes. It is a placeholder. I only added this as an implementation of Message 
since we are focusing on window operator implementation now.


> On Oct. 4, 2016, 12:10 a.m., Jake Maes wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java,
> >  li

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-10-03 Thread Jake Maes

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review151242
---


Fix it, then Ship it!




LGTM

There are still things that could be polished, but that can happen in 
subsequent patches.


samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java 
(line 137)


nit: s/Ds/Ms

There are a few instances



samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java 
(line 155)


Where is this used? I couldn't find it on any of the 3 pages of the review.



samza-operator/src/test/java/org/apache/samza/task/AssembleCallGraphTask.java 
(line 135)


This is much easier to read now. Nice!



samza-operator/src/main/java/org/apache/samza/operators/api/TriggerBuilder.java 
(line 65)


nit: I'd replace "early" with either "primary" or "regular". 

Early trigger contrasts late trigger with opposing terminology, but in 
terms of semantics, we really have a primary trigger, which is expected to 
cover the majority of the messages and then the late trigger to handle late 
arrivals. In that context, "early" doesn't make much sense because it doesn't 
sound like the normal case. 

If that^ understanding is correct, I'd suggest a rename.



samza-operator/src/main/java/org/apache/samza/operators/api/TriggerBuilder.java 
(line 139)


This is essentially the same as "addTimeoutSinceFirstMessage" with a custom 
event time function, right?

Any other differences that I'm not seeing?

No action suggested, just making sure I understand.



samza-operator/src/main/java/org/apache/samza/operators/api/TriggerBuilder.java 
(line 213)


Surprised to see these default implementations using system time rather 
than event time. Is it just because it's easier to ensure that system time 
exists and is valid?



samza-operator/src/main/java/org/apache/samza/operators/api/TriggerBuilder.java 
(line 223)


Why would one put a size limit in a late trigger rather than an early 
trigger?



samza-operator/src/main/java/org/apache/samza/operators/api/WindowState.java 
(line 47)


Why the terminology change? Here it's "earliest" and above it's "first"



samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java (line 
38)


Add a javadoc recommending a reboot if this class fails. 

Also, where's the "Start" button?

:-)



samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java (line 
68)


windowKey + tab should change applications :-)



samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java (line 
95)


What's the advantage of building the trigger here rather than before 
invoking setTriggers()?



samza-operator/src/main/java/org/apache/samza/operators/api/internal/WindowOutput.java
 (line 30)


This doesn't seem to add anything to Message. Is it just a placeholder in 
case we want to add something to window outputs and not messages? (For example, 
perhaps information about the trigger that fired.)

Is it the only implementation of Message?


- Jake Maes


On Sept. 29, 2016, 2:05 a.m., Yi Pan (Data Infrastructure) wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/47835/
> ---
> 
> (Updated Sept. 29, 2016, 2:05 a.m.)
> 
> 
> Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
> Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.
> 
> 
> Bugs: SAMZA-914
> https://issues.apache.org/jira/browse/SAMZA-914
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> SAMZA-914: initial draft of operator programming API. Design doc attached to 
> SAMZA-914: 
> https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf
> 
> 
> Diffs
> -
> 
>   build.gradle 16facbbf4dff378c561461786ff186bd9eed 
>   gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
>   samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
> 5d066c5867e9df9e94e60bde825dedf10703b399 
>   
> samza-operator/src

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-10-03 Thread Yi Pan (Data Infrastructure)


> On Oct. 3, 2016, 11:11 p.m., Jagadish Venkatraman wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java,
> >  line 102
> > 
> >
> > I thought the `Operators` class was maintaining state of the topology. 
> > (and hence, that was the reason for passing in `this` so that we know the 
> > wiring of input to output operators). Is the `MessageStream` or some other 
> > component owning that wire-up?
> > 
> > wondering why we removed `this`?

Adding this here is a bit early when we are not 100% sure what would be the 
best place to save the topology info. Hence, removing for now. I am working on 
the patch for SAMZA-915 that's to wire everything up, in which I am 
re-evaluating whether we should put it in or not.


> On Oct. 3, 2016, 11:11 p.m., Jagadish Venkatraman wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/internal/Operators.java,
> >  line 255
> > 
> >
> > nit: variable name `windowFn`. We can use `window` to refer to anything 
> > that implements a `Window`

Good point. Done.


> On Oct. 3, 2016, 11:11 p.m., Jagadish Venkatraman wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/internal/Trigger.java,
> >  line 61
> > 
> >
> > nit: *Private* constructor to prevent instantiation.

Done.


> On Oct. 3, 2016, 11:11 p.m., Jagadish Venkatraman wrote:
> > samza-operator/src/main/java/org/apache/samza/task/StreamOperatorAdaptorTask.java,
> >  line 38
> > 
> >
> > When using with HDFS-style consumer (which is bounded), how will we 
> > handle the last batch? 
> > 
> > Shouldn't the adaptor also implement the `EndOfStreamListener` task. 
> > that will guarantee that the last batch is handled correctly? (by 
> > triggering the output corresponding to the last batch).

Not in the current scope yet. I would prefer to leave it for later evaluation, 
since a) end-of-stream is not in production yet; b) this is internal class 
which does not affect user API.


> On Oct. 3, 2016, 11:11 p.m., Jagadish Venkatraman wrote:
> > samza-operator/src/test/java/org/apache/samza/task/WindowOperatorTask.java, 
> > line 50
> > 
> >
> > 1. Wouldn't it be nicer to not have an avro dependency in our 
> > open-source examples? All our open source examples are non-avro. It seems 
> > that this example maybe easy to demo with a simple `JSON`/ a `String` based 
> > example? (It can be a follow-up)
> > 
> > Is this meant to be run as a test? Or as a show-case of the API? 
> > 
> > If it's a show-case, we should move it to samza-hello-world once we're 
> > done. (can be a follow-up)
> > 
> > 2. Also, what do you think about creating a new package namespace 
> > `examples` instead of `test`?

Good point. I will remove the avro examples and replace them w/ JSON. The 
purpose of the test cases here is both for unit test and demo. We can move them 
to examples as you suggest later.


- Yi


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review151227
---


On Sept. 29, 2016, 2:05 a.m., Yi Pan (Data Infrastructure) wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/47835/
> ---
> 
> (Updated Sept. 29, 2016, 2:05 a.m.)
> 
> 
> Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
> Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.
> 
> 
> Bugs: SAMZA-914
> https://issues.apache.org/jira/browse/SAMZA-914
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> SAMZA-914: initial draft of operator programming API. Design doc attached to 
> SAMZA-914: 
> https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf
> 
> 
> Diffs
> -
> 
>   build.gradle 16facbbf4dff378c561461786ff186bd9eed 
>   gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
>   samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
> 5d066c5867e9df9e94e60bde825dedf10703b399 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStreams.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/TriggerBuilder.java
>  PRE-CREATIO

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-10-03 Thread Jagadish Venkatraman

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review151227
---



Overall, this is looking pretty good! Just some minor comments.


samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java 
(line 84)


I thought the `Operators` class was maintaining state of the topology. (and 
hence, that was the reason for passing in `this` so that we know the wiring of 
input to output operators). Is the `MessageStream` or some other component 
owning that wire-up?

wondering why we removed `this`?



samza-operator/src/main/java/org/apache/samza/operators/api/internal/Operators.java
 (line 253)


nit: variable name `windowFn`. We can use `window` to refer to anything 
that implements a `Window`



samza-operator/src/main/java/org/apache/samza/operators/api/internal/Trigger.java
 (line 61)


nit: *Private* constructor to prevent instantiation.



samza-operator/src/main/java/org/apache/samza/task/StreamOperatorAdaptorTask.java
 (line 38)


When using with HDFS-style consumer (which is bounded), how will we handle 
the last batch? 

Shouldn't the adaptor also implement the `EndOfStreamListener` task. that 
will guarantee that the last batch is handled correctly? (by triggering the 
output corresponding to the last batch).



samza-operator/src/test/java/org/apache/samza/task/WindowOperatorTask.java 
(line 49)


1. Wouldn't it be nicer to not have an avro dependency in our open-source 
examples? All our open source examples are non-avro. It seems that this example 
maybe easy to demo with a simple `JSON`/ a `String` based example? (It can be a 
follow-up)

Is this meant to be run as a test? Or as a show-case of the API? 

If it's a show-case, we should move it to samza-hello-world once we're 
done. (can be a follow-up)

2. Also, what do you think about creating a new package namespace 
`examples` instead of `test`?


- Jagadish Venkatraman


On Sept. 29, 2016, 2:05 a.m., Yi Pan (Data Infrastructure) wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/47835/
> ---
> 
> (Updated Sept. 29, 2016, 2:05 a.m.)
> 
> 
> Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
> Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.
> 
> 
> Bugs: SAMZA-914
> https://issues.apache.org/jira/browse/SAMZA-914
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> SAMZA-914: initial draft of operator programming API. Design doc attached to 
> SAMZA-914: 
> https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf
> 
> 
> Diffs
> -
> 
>   build.gradle 16facbbf4dff378c561461786ff186bd9eed 
>   gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
>   samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
> 5d066c5867e9df9e94e60bde825dedf10703b399 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStreams.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/TriggerBuilder.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/WindowState.java 
> PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/Message.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Operators.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Trigger.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/WindowFn.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/WindowOutput.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/ChainedOperators.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/ProcessorContext.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/sam

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-09-28 Thread Yi Pan (Data Infrastructure)

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/
---

(Updated Sept. 29, 2016, 2:05 a.m.)


Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.


Changes
---

Added unit tests to most of the implementation classes


Bugs: SAMZA-914
https://issues.apache.org/jira/browse/SAMZA-914


Repository: samza


Description
---

SAMZA-914: initial draft of operator programming API. Design doc attached to 
SAMZA-914: 
https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf


Diffs (updated)
-

  build.gradle 16facbbf4dff378c561461786ff186bd9eed 
  gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
  samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
5d066c5867e9df9e94e60bde825dedf10703b399 
  
samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/MessageStreams.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/TriggerBuilder.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/WindowState.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/data/Message.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/internal/Operators.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/internal/Trigger.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/internal/WindowFn.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/internal/WindowOutput.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/ChainedOperators.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/ProcessorContext.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/task/StreamOperatorAdaptorTask.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java 
PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/api/TestTriggerBuilder.java
 PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/operators/api/TestWindows.java 
PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/api/data/TestIncomingSystemMessage.java
 PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/api/data/TestLongOffset.java
 PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/api/internal/TestOperators.java
 PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/api/internal/TestTrigger.java
 PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java
 PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/impl/TestOutputMessage.java
 PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/impl/TestProcessorContext.java
 PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/impl/TestSimpleOperatorImpl.java
 PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/impl/TestSinkOperatorImpl.java
 PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/impl/TestStateStoreImpl.java
 PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/impl/window/TestSessionWindowImpl.java
 PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/AssembleCallGraphTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/BroadcastOperatorTask.java 
PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/task/InputAvroSystemMessage.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java 
PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorAdaptorTask.java
 PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorTasks.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/WindowOperatorTask.java 
PRE-CREATION 
  
samza-sql-calcite/src/test/java/org/apache/samza/sql/calcite/schema/TestAvroSchemaConverter.java
 PRE-CREATION 
  s

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-09-23 Thread Yi Pan (Data Infrastructure)

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/
---

(Updated Sept. 23, 2016, 11:52 p.m.)


Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.


Changes
---

Update the RB w/ changes earlier. More unit tests are pending.


Bugs: SAMZA-914
https://issues.apache.org/jira/browse/SAMZA-914


Repository: samza


Description
---

SAMZA-914: initial draft of operator programming API. Design doc attached to 
SAMZA-914: 
https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf


Diffs (updated)
-

  build.gradle 16facbbf4dff378c561461786ff186bd9eed 
  gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
  samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
5d066c5867e9df9e94e60bde825dedf10703b399 
  
samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/MessageStreams.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/TriggerBuilder.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/WindowState.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/data/Message.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/internal/Operators.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/internal/Trigger.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/internal/WindowFn.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/internal/WindowOutput.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/ChainedOperators.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/ProcessorContext.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/task/StreamOperatorAdaptorTask.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java 
PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/api/data/TestIncomingSystemMessage.java
 PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/api/data/TestLongOffset.java
 PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/api/internal/TestOperators.java
 PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/api/internal/TestTrigger.java
 PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/AssembleCallGraphTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/BroadcastOperatorTask.java 
PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/task/InputAvroSystemMessage.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java 
PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorTasks.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/WindowOperatorTask.java 
PRE-CREATION 
  
samza-sql-calcite/src/test/java/org/apache/samza/sql/calcite/schema/TestAvroSchemaConverter.java
 PRE-CREATION 
  samza-sql-core/README.md PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Data.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/EntityName.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Relation.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Schema.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Stream.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Table.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Tuple.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/Operator.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorCallback.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorRouter.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/Ope

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-09-23 Thread Yi Pan (Data Infrastructure)


> On Sept. 23, 2016, 9:25 p.m., Jagadish Venkatraman wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/internal/Operators.java,
> >  line 167
> > 
> >
> > How will an instance of StoreFunctions be constructed?
> > 
> > Is the intent to provide a builder/factory later?
> > 
> > The `WindowOperator` seems to initialize StoreFunctions immutably in 
> > it's constructor by setting:
> > this.storeFunctions = window.getStoreFunctions();
> > 
> > It was not clear to me how the Window class could construct an instance 
> > of `StoreFunctions`. (given that the constructor here is private)

As discussed offline, state store construction APIs are not in the scope of 
this RB yet. Feel free to add a builder/constructor method in Operators if you 
are trying to proto-type the state store APIs.


- Yi


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review150238
---


On Sept. 14, 2016, 8:53 a.m., Yi Pan (Data Infrastructure) wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/47835/
> ---
> 
> (Updated Sept. 14, 2016, 8:53 a.m.)
> 
> 
> Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
> Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.
> 
> 
> Bugs: SAMZA-914
> https://issues.apache.org/jira/browse/SAMZA-914
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> SAMZA-914: initial draft of operator programming API. Design doc attached to 
> SAMZA-914: 
> https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf
> 
> 
> Diffs
> -
> 
>   build.gradle 16facbbf4dff378c561461786ff186bd9eed 
>   gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
>   samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
> 5d066c5867e9df9e94e60bde825dedf10703b399 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStreams.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Triggers.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/WindowState.java 
> PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/Message.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/WindowOutput.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Operators.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Trigger.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Window.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/ChainedOperators.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorBaseImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/ProcessorContext.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/task/StreamOperatorAdaptorTask.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/data/TestMessageStream.java
>  PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/AssembleCallGraphTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/BroadcastOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/InputAvroSystemMessage.java
>  PRE-CREATION 
>   samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorTasks.java
>  PRE-CREATION 
>   samza-operator/src/test/java/org/apache/samza/task/WindowOperatorTask.java 
> PRE-CREATION 
>   
> samza-sql-calcite/src/test/java/org/apache/samza/sql/calcite/schema/TestAvroSchemaConverter.java
>  PRE-CREATION 
>  

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-09-23 Thread Jagadish Venkatraman

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review150238
---




samza-operator/src/main/java/org/apache/samza/operators/api/internal/Operators.java
 (line 167)


How will an instance of StoreFunctions be constructed?

Is the intent to provide a builder/factory later?

The `WindowOperator` seems to initialize StoreFunctions immutably in it's 
constructor by setting:
this.storeFunctions = window.getStoreFunctions();

It was not clear to me how the Window class could construct an instance of 
`StoreFunctions`. (given that the constructor here is private)


- Jagadish Venkatraman


On Sept. 14, 2016, 8:53 a.m., Yi Pan (Data Infrastructure) wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/47835/
> ---
> 
> (Updated Sept. 14, 2016, 8:53 a.m.)
> 
> 
> Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
> Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.
> 
> 
> Bugs: SAMZA-914
> https://issues.apache.org/jira/browse/SAMZA-914
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> SAMZA-914: initial draft of operator programming API. Design doc attached to 
> SAMZA-914: 
> https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf
> 
> 
> Diffs
> -
> 
>   build.gradle 16facbbf4dff378c561461786ff186bd9eed 
>   gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
>   samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
> 5d066c5867e9df9e94e60bde825dedf10703b399 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStreams.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Triggers.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/WindowState.java 
> PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/Message.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/WindowOutput.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Operators.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Trigger.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Window.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/ChainedOperators.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorBaseImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/ProcessorContext.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/task/StreamOperatorAdaptorTask.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/data/TestMessageStream.java
>  PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/AssembleCallGraphTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/BroadcastOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/InputAvroSystemMessage.java
>  PRE-CREATION 
>   samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorTasks.java
>  PRE-CREATION 
>   samza-operator/src/test/java/org/apache/samza/task/WindowOperatorTask.java 
> PRE-CREATION 
>   
> samza-sql-calcite/src/test/java/org/apache/samza/sql/calcite/schema/TestAvroSchemaConverter.java
>  PRE-CREATION 
>   samza-sql-core/README.md PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Data.java 
> PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/samza/sql/api/data/EntityName.java 
> PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Relation.java 
> PRE-CREATION 
>   sa

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-09-21 Thread Yi Pan (Data Infrastructure)


> On Sept. 20, 2016, 8:35 p.m., Jagadish Venkatraman wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java, 
> > line 127
> > 
> >
> > Currently we have 2 flavors of the SessionWindow API:
> > 
> > 1. One for tasks like count(), where the session function is `(m,M) -> 
> > m+1`. 
> > 2. Other for arbitrary aggregates for which the sessionFunction is 
> > `(m,M) -> {M.append(m)}`. In this case, the value stored is a collection of 
> > `` or a collection of ``
> > 
> > I wonder if we could expose a flavor of this API (similar to a reduce) 
> > that takes in a custom `aggregationFunction`. 
> > 
> > `public static  SessionWindow 
> > intoSessions(Function  sessionKeyFunction, AGG initialValue, Function 
> >  aggregatorFunction)`
> > 
> > You would not need to store the entire list of objects in the session 
> > but can only store the AGG result. It's possible to do this by 
> > instantiating a new `SessionWindow` object but, it maybe a good idea to 
> > expose this as a builder in `Windows`.
> > 
> > Let me know what you think.

Discussed offline. This can be a simple addition to the existing API. The 
current two flavors are representative cases of a) single aggregated partial 
result per window; b) full set of messages as partial result per window. The 
customized partial aggregated result also need to be included in the state 
store design, which I purposefully left out of the scope of the current RB. We 
can add the customized aggregate function easily when we figure out all to 
allow users to customize window operator's state later.


- Yi


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review149724
---


On Sept. 14, 2016, 8:53 a.m., Yi Pan (Data Infrastructure) wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/47835/
> ---
> 
> (Updated Sept. 14, 2016, 8:53 a.m.)
> 
> 
> Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
> Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.
> 
> 
> Bugs: SAMZA-914
> https://issues.apache.org/jira/browse/SAMZA-914
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> SAMZA-914: initial draft of operator programming API. Design doc attached to 
> SAMZA-914: 
> https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf
> 
> 
> Diffs
> -
> 
>   build.gradle 16facbbf4dff378c561461786ff186bd9eed 
>   gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
>   samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
> 5d066c5867e9df9e94e60bde825dedf10703b399 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStreams.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Triggers.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/WindowState.java 
> PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/Message.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/WindowOutput.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Operators.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Trigger.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Window.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/ChainedOperators.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorBaseImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/ProcessorContext.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/task/StreamOperatorAdaptorTask.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apa

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-09-21 Thread Yi Pan (Data Infrastructure)


> On Sept. 20, 2016, 8:35 p.m., Jagadish Venkatraman wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java, 
> > line 127
> > 
> >
> > Currently we have 2 flavors of the SessionWindow API:
> > 
> > 1. One for tasks like count(), where the session function is `(m,M) -> 
> > m+1`. 
> > 2. Other for arbitrary aggregates for which the sessionFunction is 
> > `(m,M) -> {M.append(m)}`. In this case, the value stored is a collection of 
> > `` or a collection of ``
> > 
> > I wonder if we could expose a flavor of this API (similar to a reduce) 
> > that takes in a custom `aggregationFunction`. 
> > 
> > `public static  SessionWindow 
> > intoSessions(Function  sessionKeyFunction, AGG initialValue, Function 
> >  aggregatorFunction)`
> > 
> > You would not need to store the entire list of objects in the session 
> > but can only store the AGG result. It's possible to do this by 
> > instantiating a new `SessionWindow` object but, it maybe a good idea to 
> > expose this as a builder in `Windows`.
> > 
> > Let me know what you think.
> 
> Yi Pan (Data Infrastructure) wrote:
> Discussed offline. This can be a simple addition to the existing API. The 
> current two flavors are representative cases of a) single aggregated partial 
> result per window; b) full set of messages as partial result per window. The 
> customized partial aggregated result also need to be included in the state 
> store design, which I purposefully left out of the scope of the current RB. 
> We can add the customized aggregate function easily when we figure out all to 
> allow users to customize window operator's state later.

P.S. the current APIs also allows customized aggregation as well, via a 
SessionWindow + map that applies the customized aggregate function in map on 
top of the output message from the SessionWindow.


- Yi


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review149724
---


On Sept. 14, 2016, 8:53 a.m., Yi Pan (Data Infrastructure) wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/47835/
> ---
> 
> (Updated Sept. 14, 2016, 8:53 a.m.)
> 
> 
> Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
> Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.
> 
> 
> Bugs: SAMZA-914
> https://issues.apache.org/jira/browse/SAMZA-914
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> SAMZA-914: initial draft of operator programming API. Design doc attached to 
> SAMZA-914: 
> https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf
> 
> 
> Diffs
> -
> 
>   build.gradle 16facbbf4dff378c561461786ff186bd9eed 
>   gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
>   samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
> 5d066c5867e9df9e94e60bde825dedf10703b399 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStreams.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Triggers.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/WindowState.java 
> PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/Message.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/WindowOutput.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Operators.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Trigger.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Window.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/ChainedOperators.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorBaseImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/ProcessorContext.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
> 

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-09-20 Thread Jagadish Venkatraman

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review149724
---




samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java (line 
127)


Currently we have 2 flavors of the SessionWindow API:

1. One for tasks like count(), where the session function is `(m,M) -> 
m+1`. 
2. Other for arbitrary aggregates for which the sessionFunction is `(m,M) 
-> {M.append(m)}`. In this case, the value stored is a collection of `` or a 
collection of ``

I wonder if we could expose a flavor of this API (similar to a reduce) that 
takes in a custom `aggregationFunction`. 

`public static  SessionWindow 
intoSessions(Function  sessionKeyFunction, AGG initialValue, Function 
 aggregatorFunction)`

You would not need to store the entire list of objects in the session but 
can only store the AGG result. It's possible to do this by instantiating a new 
`SessionWindow` object but, it maybe a good idea to expose this as a builder in 
`Windows`.

Let me know what you think.


- Jagadish Venkatraman


On Sept. 14, 2016, 8:53 a.m., Yi Pan (Data Infrastructure) wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/47835/
> ---
> 
> (Updated Sept. 14, 2016, 8:53 a.m.)
> 
> 
> Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
> Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.
> 
> 
> Bugs: SAMZA-914
> https://issues.apache.org/jira/browse/SAMZA-914
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> SAMZA-914: initial draft of operator programming API. Design doc attached to 
> SAMZA-914: 
> https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf
> 
> 
> Diffs
> -
> 
>   build.gradle 16facbbf4dff378c561461786ff186bd9eed 
>   gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
>   samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
> 5d066c5867e9df9e94e60bde825dedf10703b399 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStreams.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Triggers.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/WindowState.java 
> PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/Message.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/WindowOutput.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Operators.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Trigger.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Window.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/ChainedOperators.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorBaseImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/ProcessorContext.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/task/StreamOperatorAdaptorTask.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/data/TestMessageStream.java
>  PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/AssembleCallGraphTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/BroadcastOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/InputAvroSystemMessage.java
>  PRE-CREATION 
>   samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorTasks.java
>  PRE-CREATION 
>   samza-operator/src/test/java/org/apache/samza/task/WindowOperatorTask.java 
> PRE-CREATION 
>   
> samza-sq

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-09-19 Thread Yi Pan (Data Infrastructure)


> On Sept. 19, 2016, 9:24 p.m., Jagadish Venkatraman wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/internal/Operators.java,
> >  line 236
> > 
> >
> > nit: 
> > Small typo here - Maybe, rename this to `getTransformFunc()` or 
> > `getFunction()` .

Sure. Will do.


- Yi


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review149545
---


On Sept. 14, 2016, 8:53 a.m., Yi Pan (Data Infrastructure) wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/47835/
> ---
> 
> (Updated Sept. 14, 2016, 8:53 a.m.)
> 
> 
> Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
> Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.
> 
> 
> Bugs: SAMZA-914
> https://issues.apache.org/jira/browse/SAMZA-914
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> SAMZA-914: initial draft of operator programming API. Design doc attached to 
> SAMZA-914: 
> https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf
> 
> 
> Diffs
> -
> 
>   build.gradle 16facbbf4dff378c561461786ff186bd9eed 
>   gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
>   samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
> 5d066c5867e9df9e94e60bde825dedf10703b399 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStreams.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Triggers.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/WindowState.java 
> PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/Message.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/WindowOutput.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Operators.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Trigger.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Window.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/ChainedOperators.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorBaseImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/ProcessorContext.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/task/StreamOperatorAdaptorTask.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/data/TestMessageStream.java
>  PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/AssembleCallGraphTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/BroadcastOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/InputAvroSystemMessage.java
>  PRE-CREATION 
>   samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorTasks.java
>  PRE-CREATION 
>   samza-operator/src/test/java/org/apache/samza/task/WindowOperatorTask.java 
> PRE-CREATION 
>   
> samza-sql-calcite/src/test/java/org/apache/samza/sql/calcite/schema/TestAvroSchemaConverter.java
>  PRE-CREATION 
>   samza-sql-core/README.md PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Data.java 
> PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/samza/sql/api/data/EntityName.java 
> PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Relation.java 
> PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Schema.java 
> PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Stream.java 
> PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-09-19 Thread Jagadish Venkatraman

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review149545
---




samza-operator/src/main/java/org/apache/samza/operators/api/internal/Operators.java
 (line 236)


nit: 
Small typo here - Maybe, rename this to `getTransformFunc()` or 
`getFunction()` .


- Jagadish Venkatraman


On Sept. 14, 2016, 8:53 a.m., Yi Pan (Data Infrastructure) wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/47835/
> ---
> 
> (Updated Sept. 14, 2016, 8:53 a.m.)
> 
> 
> Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
> Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.
> 
> 
> Bugs: SAMZA-914
> https://issues.apache.org/jira/browse/SAMZA-914
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> SAMZA-914: initial draft of operator programming API. Design doc attached to 
> SAMZA-914: 
> https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf
> 
> 
> Diffs
> -
> 
>   build.gradle 16facbbf4dff378c561461786ff186bd9eed 
>   gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
>   samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
> 5d066c5867e9df9e94e60bde825dedf10703b399 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStreams.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Triggers.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/WindowState.java 
> PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/Message.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/WindowOutput.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Operators.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Trigger.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Window.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/ChainedOperators.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorBaseImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/ProcessorContext.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/task/StreamOperatorAdaptorTask.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/data/TestMessageStream.java
>  PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/AssembleCallGraphTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/BroadcastOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/InputAvroSystemMessage.java
>  PRE-CREATION 
>   samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorTasks.java
>  PRE-CREATION 
>   samza-operator/src/test/java/org/apache/samza/task/WindowOperatorTask.java 
> PRE-CREATION 
>   
> samza-sql-calcite/src/test/java/org/apache/samza/sql/calcite/schema/TestAvroSchemaConverter.java
>  PRE-CREATION 
>   samza-sql-core/README.md PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Data.java 
> PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/samza/sql/api/data/EntityName.java 
> PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Relation.java 
> PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Schema.java 
> PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Stream.java 
> PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Table.java 
> PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/samza/sql/api/data

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-09-18 Thread Yi Pan (Data Infrastructure)


> On Sept. 14, 2016, 7:03 p.m., Chris Pettitt wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/internal/Window.java,
> >  line 39
> > 
> >
> > IIUC given this interface Window is basically an entirely opaque object 
> > to the client code?

Replaced w/ WindowFn interface class. And yeah, this should not be seen by the 
client code.


- Yi


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review148908
---


On Sept. 14, 2016, 8:53 a.m., Yi Pan (Data Infrastructure) wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/47835/
> ---
> 
> (Updated Sept. 14, 2016, 8:53 a.m.)
> 
> 
> Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
> Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.
> 
> 
> Bugs: SAMZA-914
> https://issues.apache.org/jira/browse/SAMZA-914
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> SAMZA-914: initial draft of operator programming API. Design doc attached to 
> SAMZA-914: 
> https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf
> 
> 
> Diffs
> -
> 
>   build.gradle 16facbbf4dff378c561461786ff186bd9eed 
>   gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
>   samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
> 5d066c5867e9df9e94e60bde825dedf10703b399 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStreams.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Triggers.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/WindowState.java 
> PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/Message.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/WindowOutput.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Operators.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Trigger.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Window.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/ChainedOperators.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorBaseImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/ProcessorContext.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/task/StreamOperatorAdaptorTask.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/data/TestMessageStream.java
>  PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/AssembleCallGraphTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/BroadcastOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/InputAvroSystemMessage.java
>  PRE-CREATION 
>   samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorTasks.java
>  PRE-CREATION 
>   samza-operator/src/test/java/org/apache/samza/task/WindowOperatorTask.java 
> PRE-CREATION 
>   
> samza-sql-calcite/src/test/java/org/apache/samza/sql/calcite/schema/TestAvroSchemaConverter.java
>  PRE-CREATION 
>   samza-sql-core/README.md PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Data.java 
> PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/samza/sql/api/data/EntityName.java 
> PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Relation.java 
> PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Schema.java 
> PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Stream.java

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-09-18 Thread Yi Pan (Data Infrastructure)


> On Sept. 14, 2016, 7:03 p.m., Chris Pettitt wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java, 
> > line 48
> > 
> >
> > Does the concrete class need to be exposed? Interfaces are generally 
> > nicer to program against for extensibility and testing. Though in this case 
> > it looks like Window is ABC not an interface.
> 
> Yi Pan (Data Infrastructure) wrote:
> Window is an ABC, and is intended to be used just by internal and 
> implementation classes. I will add a public interface class as a return 
> class. Thanks for the suggestion.

Replaced it w/ two interface classes: Window as a public interface class that 
programmer gets from Windows, while added WindowFn interface class in 
operator.api.internal package for methods that are only used by 
representation/implementation classes. Each concrete window class will 
implement the public Window interface and Windows.getInternalWindowFn(window) 
will instantiate a cooresponding internal interface (WindowFn) object to be 
used by representation/implementation classes.


- Yi


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review148908
---


On Sept. 14, 2016, 8:53 a.m., Yi Pan (Data Infrastructure) wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/47835/
> ---
> 
> (Updated Sept. 14, 2016, 8:53 a.m.)
> 
> 
> Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
> Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.
> 
> 
> Bugs: SAMZA-914
> https://issues.apache.org/jira/browse/SAMZA-914
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> SAMZA-914: initial draft of operator programming API. Design doc attached to 
> SAMZA-914: 
> https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf
> 
> 
> Diffs
> -
> 
>   build.gradle 16facbbf4dff378c561461786ff186bd9eed 
>   gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
>   samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
> 5d066c5867e9df9e94e60bde825dedf10703b399 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStreams.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Triggers.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/WindowState.java 
> PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/Message.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/WindowOutput.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Operators.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Trigger.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Window.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/ChainedOperators.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorBaseImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/ProcessorContext.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/task/StreamOperatorAdaptorTask.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/data/TestMessageStream.java
>  PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/AssembleCallGraphTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/BroadcastOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/InputAvroSystemMessage.java
>  PRE-CREATION 
>   samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorTasks.java
>  PR

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-09-16 Thread Yi Pan (Data Infrastructure)


> On Sept. 14, 2016, 7:03 p.m., Chris Pettitt wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java,
> >  line 53
> > 
> >
> > Minor: you have inline other transform functions below, so it's not 
> > obvious why this was not inlined. Inlined might be slightly nicer (for 
> > review at least :)) as it keeps the code in one place. Your call.

Make sense. I added this simply for code re-use purpose. Since there are not 
many places referring to this function, inline makes sense.


> On Sept. 14, 2016, 7:03 p.m., Chris Pettitt wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java,
> >  line 67
> > 
> >
> > Doc and code do not agree. The code has unbounded type parameters, 
> > while the doc suggests specific types.
> > 
> > If the doc is correct, we should be able to satisfy it without a 
> > generic type.

Good catch! Thanks for pointing out this. Will fix and do a scrub through the 
code for doc/code consistency.


> On Sept. 14, 2016, 7:03 p.m., Chris Pettitt wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/Triggers.java, 
> > line 30
> > 
> >
> > It's probably worth noting that if there are multiple triggers the 
> > aggregate value is the disjunction of the individual values.

Makes sense. Will add to the doc. Thanks!


> On Sept. 14, 2016, 7:03 p.m., Chris Pettitt wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/Triggers.java, 
> > line 39
> > 
> >
> > Maybe TriggerBuilder as this differs a bit from the typical class of 
> > static factory methods implied by the class name?

Sure. This is actually a builder.


> On Sept. 14, 2016, 7:03 p.m., Chris Pettitt wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java, 
> > line 48
> > 
> >
> > Does the concrete class need to be exposed? Interfaces are generally 
> > nicer to program against for extensibility and testing. Though in this case 
> > it looks like Window is ABC not an interface.

Window is an ABC, and is intended to be used just by internal and 
implementation classes. I will add a public interface class as a return class. 
Thanks for the suggestion.


> On Sept. 14, 2016, 7:03 p.m., Chris Pettitt wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/internal/Operators.java,
> >  line 44
> > 
> >
> > You can use a private constructor to prevent subclassing / 
> > instantiation.

Make sense! Will do. The same for MessageStreams.


> On Sept. 14, 2016, 7:03 p.m., Chris Pettitt wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/internal/Operators.java,
> >  line 332
> > 
> >
> > Can you drop input here as it is actually unused? I found my way here 
> > due to seeing something else handing this the "this" pointer, which looked 
> > suspicious. Turns out it is not used - maybe for type parameter bounds 
> > checking? but is so, I'd be interested to see if we could find a more 
> > direct and obvious approach.
> > 
> > ---
> > 
> > Also, as far as I can tell, this looks like it can be moved straight 
> > into MessageStream (perhaps as a private method). I didn't see anything 
> > else using it, but maybe I missed it.

As we discussed, I will drop it for now and revisit it in the later patch that 
implements the execution graph.


> On Sept. 14, 2016, 7:03 p.m., Chris Pettitt wrote:
> > samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java, 
> > line 39
> > 
> >
> > Minor: abstract not necessary here. (Technically public isn't either 
> > :)).

Yeah, it is a copy/paste problem when move this method to a separate interface 
class. Thanks for pointing out!


> On Sept. 14, 2016, 7:03 p.m., Chris Pettitt wrote:
> > samza-operator/src/test/java/org/apache/samza/operators/api/data/TestMessageStream.java,
> >  lines 42-61
> > 
> >
> > This doesn't appear to be testing anything (other than that types 
> > check).

This RB is limited to APIs. I will add more assert/test code to ensure the 
internal variable/methods are setup/invoked correctly.


> On Sept. 14, 2016, 7:03 p.m., Chris Pettitt wrote:
> > samza-operator/src/test/java/org/apache/samza/task/AssembleCallGraphTask.java,
> >  line 141
> > 

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-09-14 Thread Chris Pettitt


> On Sept. 14, 2016, 7:03 p.m., Chris Pettitt wrote:
> > This review turned out not to be so massive as I expected (lots of 
> > deletes). I don't see any serious issues. There are some minor cosmetic 
> > issues and some subjective stuff that you can take or leave. I tried to not 
> > mark subjective stuff as an issue.

BTW, I should have mentioned that this is looking pretty good. Nice work!


- Chris


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review148908
---


On Sept. 14, 2016, 8:53 a.m., Yi Pan (Data Infrastructure) wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/47835/
> ---
> 
> (Updated Sept. 14, 2016, 8:53 a.m.)
> 
> 
> Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
> Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.
> 
> 
> Bugs: SAMZA-914
> https://issues.apache.org/jira/browse/SAMZA-914
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> SAMZA-914: initial draft of operator programming API. Design doc attached to 
> SAMZA-914: 
> https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf
> 
> 
> Diffs
> -
> 
>   build.gradle 16facbbf4dff378c561461786ff186bd9eed 
>   gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
>   samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
> 5d066c5867e9df9e94e60bde825dedf10703b399 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStreams.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Triggers.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/WindowState.java 
> PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/Message.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/WindowOutput.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Operators.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Trigger.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Window.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/ChainedOperators.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorBaseImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/ProcessorContext.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/task/StreamOperatorAdaptorTask.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/data/TestMessageStream.java
>  PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/AssembleCallGraphTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/BroadcastOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/InputAvroSystemMessage.java
>  PRE-CREATION 
>   samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorTasks.java
>  PRE-CREATION 
>   samza-operator/src/test/java/org/apache/samza/task/WindowOperatorTask.java 
> PRE-CREATION 
>   
> samza-sql-calcite/src/test/java/org/apache/samza/sql/calcite/schema/TestAvroSchemaConverter.java
>  PRE-CREATION 
>   samza-sql-core/README.md PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Data.java 
> PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/samza/sql/api/data/EntityName.java 
> PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Relation.java 
> PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Schema.java 
> PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Stream.java 
> PRE-CREATION 
>   samza-sql-core

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-09-14 Thread Chris Pettitt

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review148908
---



This review turned out not to be so massive as I expected (lots of deletes). I 
don't see any serious issues. There are some minor cosmetic issues and some 
subjective stuff that you can take or leave. I tried to not mark subjective 
stuff as an issue.


samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java 
(line 53)


Minor: you have inline other transform functions below, so it's not obvious 
why this was not inlined. Inlined might be slightly nicer (for review at least 
:)) as it keeps the code in one place. Your call.



samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java 
(line 67)


Doc and code do not agree. The code has unbounded type parameters, while 
the doc suggests specific types.

If the doc is correct, we should be able to satisfy it without a generic 
type.



samza-operator/src/main/java/org/apache/samza/operators/api/MessageStreams.java 
(line 32)


Private constructor. I explain the benefits either above or below (been 
jumping around a bit).



samza-operator/src/main/java/org/apache/samza/operators/api/Triggers.java (line 
30)


It's probably worth noting that if there are multiple triggers the 
aggregate value is the disjunction of the individual values.



samza-operator/src/main/java/org/apache/samza/operators/api/Triggers.java (line 
39)


Maybe TriggerBuilder as this differs a bit from the typical class of static 
factory methods implied by the class name?



samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java (line 
48)


Does the concrete class need to be exposed? Interfaces are generally nicer 
to program against for extensibility and testing. Though in this case it looks 
like Window is ABC not an interface.



samza-operator/src/main/java/org/apache/samza/operators/api/internal/Operators.java
 (line 44)


You can use a private constructor to prevent subclassing / instantiation.



samza-operator/src/main/java/org/apache/samza/operators/api/internal/Operators.java
 (line 332)


Can you drop input here as it is actually unused? I found my way here due 
to seeing something else handing this the "this" pointer, which looked 
suspicious. Turns out it is not used - maybe for type parameter bounds 
checking? but is so, I'd be interested to see if we could find a more direct 
and obvious approach.

---

Also, as far as I can tell, this looks like it can be moved straight into 
MessageStream (perhaps as a private method). I didn't see anything else using 
it, but maybe I missed it.



samza-operator/src/main/java/org/apache/samza/operators/api/internal/Window.java
 (line 39)


IIUC given this interface Window is basically an entirely opaque object to 
the client code?



samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java 
(line 39)


Minor: abstract not necessary here. (Technically public isn't either :)).



samza-operator/src/test/java/org/apache/samza/operators/api/data/TestMessageStream.java
 (lines 42 - 61)


This doesn't appear to be testing anything (other than that types check).



samza-operator/src/test/java/org/apache/samza/task/AssembleCallGraphTask.java 
(line 141)


Minor: indentation is off here.


- Chris Pettitt


On Sept. 14, 2016, 8:53 a.m., Yi Pan (Data Infrastructure) wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/47835/
> ---
> 
> (Updated Sept. 14, 2016, 8:53 a.m.)
> 
> 
> Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
> Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.
> 
> 
> Bugs: SAMZA-914
> https://issues.apache.org/jira/browse/SAMZA-914
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> SAMZA-914: initial draft of operator programming API. Design doc attached to 
> SAMZA-914: 
> https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf
> 
> 
> Diffs
> -
> 
>   bu

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-09-14 Thread Yi Pan (Data Infrastructure)

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/
---

(Updated Sept. 14, 2016, 8:53 a.m.)


Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.


Changes
---

Addressing Jagadish's comments. Made the following changes:
- removing Join and replaced w/ MessageStream.join()
- added MessageStream.merge()
- updated the StreamOperatorTask.initOperators() to take all input system 
streams as parameter and only invoke once
- updated tests and examples


Bugs: SAMZA-914
https://issues.apache.org/jira/browse/SAMZA-914


Repository: samza


Description
---

SAMZA-914: initial draft of operator programming API. Design doc attached to 
SAMZA-914: 
https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf


Diffs (updated)
-

  build.gradle 16facbbf4dff378c561461786ff186bd9eed 
  gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
  samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
5d066c5867e9df9e94e60bde825dedf10703b399 
  
samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/MessageStreams.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/Triggers.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/WindowState.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/data/Message.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/data/WindowOutput.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/internal/Operators.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/internal/Trigger.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/internal/Window.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/ChainedOperators.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorBaseImpl.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/ProcessorContext.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/task/StreamOperatorAdaptorTask.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java 
PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/api/data/TestMessageStream.java
 PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/AssembleCallGraphTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/BroadcastOperatorTask.java 
PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/task/InputAvroSystemMessage.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java 
PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorTasks.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/WindowOperatorTask.java 
PRE-CREATION 
  
samza-sql-calcite/src/test/java/org/apache/samza/sql/calcite/schema/TestAvroSchemaConverter.java
 PRE-CREATION 
  samza-sql-core/README.md PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Data.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/EntityName.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Relation.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Schema.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Stream.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Table.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Tuple.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/Operator.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorCallback.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorRouter.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorSpec.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SimpleOp

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-09-14 Thread Yi Pan (Data Infrastructure)


> On Sept. 13, 2016, 10:03 p.m., Jagadish Venkatraman wrote:
> > samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java, 
> > line 48
> > 
> >
> > Curious as to why should users rely on `Join.join` to do the join? I 
> > wonder if we could directly invoke a
> > `joinSource.join(joinSource2)`? 
> > 
> > That way, I can do a more fluent groupBy on a join result?
> > 
> > joinSource.join(joinSource2)
> >   .join(joinSource3)
> >   .window(SessionWindows.into ())
> >   .sink(SinkFunction)
> 
> Yi Pan (Data Infrastructure) wrote:
> As we discussed offline, there are more details in why the above join() 
> does not play well w/ the generic type parameters to ensure the same key in 
> both streams in a join and typed joiner functions in the two 
> stream-to-buffered-stream joins are also typed. I agree that this Join.join 
> API is not the most intuitive way to me either. Besides, the current 
> MessageStream.join() method is package private and not a public API. We can 
> also add to the public API by making a better version of MessageStream.join() 
> public. For now, let's pond on it a bit more before making the change.

I made an attempt to change it to MessageStream.join(), w/ the assumption that 
the K in Message is the join key. It looks good, except that type K is 
not guaranteed as the same as the type of key in M as in MessageStream. 
Since it is only one exception here, I would make the tradeoff to let this one 
go. Please take a look to see whether that is better. Thanks for the suggestion!


- Yi


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review148787
---


On Sept. 12, 2016, 5:53 p.m., Yi Pan (Data Infrastructure) wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/47835/
> ---
> 
> (Updated Sept. 12, 2016, 5:53 p.m.)
> 
> 
> Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
> Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.
> 
> 
> Bugs: SAMZA-914
> https://issues.apache.org/jira/browse/SAMZA-914
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> SAMZA-914: initial draft of operator programming API. Design doc attached to 
> SAMZA-914: 
> https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf
> 
> 
> Diffs
> -
> 
>   build.gradle 16facbbf4dff378c561461786ff186bd9eed 
>   gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
>   samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
> 5d066c5867e9df9e94e60bde825dedf10703b399 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Join.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStreams.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Triggers.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/WindowState.java 
> PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/Message.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/WindowOutput.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Operators.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Trigger.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Window.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/ChainedOperators.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorBaseImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/ProcessorContext.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/task/StreamOperatorAdaptorTask.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java 
> PRE-

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-09-14 Thread Yi Pan (Data Infrastructure)


> On Sept. 13, 2016, 10:03 p.m., Jagadish Venkatraman wrote:
> > samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java, 
> > line 47
> > 
> >
> > The current implementation seems to rely on state maintained across 
> > multiple invocations of `initOperators` to correctly invoke the join 
> > function.
> > 
> > Suggestion 1:
> > I wonder if we could pass in list of all `SystemMessageStreams` in this 
> > method? (Or perhaps - declaratively create a `MessageStream` in code doing 
> > a `new MessageStream(systemStreamPartition);`? 
> > 
> > That way users can do a one-shot initialization? 
> > 
> > Suggestion 2:
> > What if we just pass in a `List` objects and 
> > users can choose to instantiate `messageStreams` and register them. 
> > 
> > Please let me know if I don't fully understand the constraints of the 
> > design!

I will update the test case based on the updated initOperators() method. Thanks!


> On Sept. 13, 2016, 10:03 p.m., Jagadish Venkatraman wrote:
> > samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java, 
> > line 48
> > 
> >
> > Curious as to why should users rely on `Join.join` to do the join? I 
> > wonder if we could directly invoke a
> > `joinSource.join(joinSource2)`? 
> > 
> > That way, I can do a more fluent groupBy on a join result?
> > 
> > joinSource.join(joinSource2)
> >   .join(joinSource3)
> >   .window(SessionWindows.into ())
> >   .sink(SinkFunction)

As we discussed offline, there are more details in why the above join() does 
not play well w/ the generic type parameters to ensure the same key in both 
streams in a join and typed joiner functions in the two 
stream-to-buffered-stream joins are also typed. I agree that this Join.join API 
is not the most intuitive way to me either. Besides, the current 
MessageStream.join() method is package private and not a public API. We can 
also add to the public API by making a better version of MessageStream.join() 
public. For now, let's pond on it a bit more before making the change.


- Yi


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review148787
---


On Sept. 12, 2016, 5:53 p.m., Yi Pan (Data Infrastructure) wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/47835/
> ---
> 
> (Updated Sept. 12, 2016, 5:53 p.m.)
> 
> 
> Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
> Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.
> 
> 
> Bugs: SAMZA-914
> https://issues.apache.org/jira/browse/SAMZA-914
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> SAMZA-914: initial draft of operator programming API. Design doc attached to 
> SAMZA-914: 
> https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf
> 
> 
> Diffs
> -
> 
>   build.gradle 16facbbf4dff378c561461786ff186bd9eed 
>   gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
>   samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
> 5d066c5867e9df9e94e60bde825dedf10703b399 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Join.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStreams.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Triggers.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/WindowState.java 
> PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/Message.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/WindowOutput.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Operators.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Trigger.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Window.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/ChainedOperators.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorBaseImpl.java
>  PRE-CREATION 
>   
> samza-operato

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-09-14 Thread Yi Pan (Data Infrastructure)


> On Sept. 13, 2016, 10:03 p.m., Jagadish Venkatraman wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/MessageStreams.java,
> >  line 37
> > 
> >
> > What do you think about adding a getSystemStreamPartition() function in 
> > the `SystemMessageStream` class?
> > 
> > The following maybe an issue with the current design:
> > 
> > In the `initOperator` callback that the user receives, how will a user 
> > know what SSP that a `SystemMessageStream` is for?
> > 
> > For example, depending on what stream I'm consuming I would want to 
> > initialize them / perform different transforms on them.

Yep! Thanks for trying out more on the join case. The suggestion totally make 
sense. Done!


> On Sept. 13, 2016, 10:03 p.m., Jagadish Venkatraman wrote:
> > samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java, 
> > line 40
> > 
> >
> > I wonder if there's a better way to express the 'init' of the sources 
> > (instead of providing a  callback that contains `a` single `MessageStream`.

Good suggestion! I have made the change.


- Yi


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review148787
---


On Sept. 12, 2016, 5:53 p.m., Yi Pan (Data Infrastructure) wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/47835/
> ---
> 
> (Updated Sept. 12, 2016, 5:53 p.m.)
> 
> 
> Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
> Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.
> 
> 
> Bugs: SAMZA-914
> https://issues.apache.org/jira/browse/SAMZA-914
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> SAMZA-914: initial draft of operator programming API. Design doc attached to 
> SAMZA-914: 
> https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf
> 
> 
> Diffs
> -
> 
>   build.gradle 16facbbf4dff378c561461786ff186bd9eed 
>   gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
>   samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
> 5d066c5867e9df9e94e60bde825dedf10703b399 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Join.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStreams.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Triggers.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/WindowState.java 
> PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/Message.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/WindowOutput.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Operators.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Trigger.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Window.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/ChainedOperators.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorBaseImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/ProcessorContext.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/task/StreamOperatorAdaptorTask.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/data/TestMessageStream.java
>  PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/AssembleCallGraphTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/BroadcastOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/InputAvroSystemMessage.java
>  PRE-CRE

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-09-13 Thread Jagadish Venkatraman

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review148787
---



I started prototyping the following usecase:

` * A task that joins PageViewEvent {userId, pageUrl, timeStamp} with the 
UserProfile stream {userId, region}, and then counts the number of
  * page views by region. It outputs a kafka message {region, pageUrl, count}`

and have my initial set of comments. I will follow up with you offline.


samza-operator/src/main/java/org/apache/samza/operators/api/MessageStreams.java 
(line 37)


What do you think about adding a getSystemStreamPartition() function in the 
`SystemMessageStream` class?

The following maybe an issue with the current design:

In the `initOperator` callback that the user receives, how will a user know 
what SSP that a `SystemMessageStream` is for?

For example, depending on what stream I'm consuming I would want to 
initialize them / perform different transforms on them.



samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java (line 
40)


I wonder if there's a better way to express the 'init' of the sources 
(instead of providing a  callback that contains `a` single `MessageStream`.



samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java (line 
47)


The current implementation seems to rely on state maintained across 
multiple invocations of `initOperators` to correctly invoke the join function.

Suggestion 1:
I wonder if we could pass in list of all `SystemMessageStreams` in this 
method? (Or perhaps - declaratively create a `MessageStream` in code doing a 
`new MessageStream(systemStreamPartition);`? 

That way users can do a one-shot initialization? 

Suggestion 2:
What if we just pass in a `List` objects and users 
can choose to instantiate `messageStreams` and register them. 

Please let me know if I don't fully understand the constraints of the 
design!



samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java (line 
48)


Curious as to why should users rely on `Join.join` to do the join? I wonder 
if we could directly invoke a
`joinSource.join(joinSource2)`? 

That way, I can do a more fluent groupBy on a join result?

joinSource.join(joinSource2)
  .join(joinSource3)
  .window(SessionWindows.into ())
  .sink(SinkFunction)


- Jagadish Venkatraman


On Sept. 12, 2016, 5:53 p.m., Yi Pan (Data Infrastructure) wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/47835/
> ---
> 
> (Updated Sept. 12, 2016, 5:53 p.m.)
> 
> 
> Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
> Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.
> 
> 
> Bugs: SAMZA-914
> https://issues.apache.org/jira/browse/SAMZA-914
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> SAMZA-914: initial draft of operator programming API. Design doc attached to 
> SAMZA-914: 
> https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf
> 
> 
> Diffs
> -
> 
>   build.gradle 16facbbf4dff378c561461786ff186bd9eed 
>   gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
>   samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
> 5d066c5867e9df9e94e60bde825dedf10703b399 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Join.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStreams.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Triggers.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/WindowState.java 
> PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/Message.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/WindowOutput.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Operators.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Trigger.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/internal/Win

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-09-12 Thread Yi Pan (Data Infrastructure)


> On Aug. 26, 2016, 7:34 p.m., Chris Pettitt wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/window/Trigger.java,
> >  line 28
> > 
> >
> > Definitely not digging the mutable state here.
> > 
> > Trigger a = Trigger.of(...);
> > Trigger b = Trigger.of(...);
> > Trigger c = Trigger.of(...);
> > 
> > Trigger ab = a.and(b);
> > Trigger ac = a.and(c);
> > 
> > Now both ab and ac are really a and b and c. Imagine if this weren't 
> > all in one function!
> > 
> > It appears that Trigger may not actually be necessary. Essentially this 
> > class is just a set of helpers for building predicates. I could imagine 
> > something like this:
> > 
> > ```
> > public static class PredicateHelper {
> > public static  Function and(Function 
> > lhs, Function rhs) {
> > return new Function() {
> > @Override
> > public Boolean apply(S s) {
> > return lhs.apply(s) && rhs.apply(s);
> > }
> > };
> > }
> > }
> > ```
> > 
> > Main point though is to use immutability.
> 
> Yi Pan (Data Infrastructure) wrote:
> You are completely right that this is a way to build predicates. I will 
> try this out w/ the new Trigger API updates.

Added Triggers which include all mutable methods that are used by programmers 
directly, and an immutable Trigger class as the implementation from 
Triggers.build().


- Yi


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review147014
---


On Aug. 26, 2016, 8:43 p.m., Yi Pan (Data Infrastructure) wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/47835/
> ---
> 
> (Updated Aug. 26, 2016, 8:43 p.m.)
> 
> 
> Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
> Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.
> 
> 
> Bugs: SAMZA-914
> https://issues.apache.org/jira/browse/SAMZA-914
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> SAMZA-914: initial draft of operator programming API. Design doc attached to 
> SAMZA-914: 
> https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf
> 
> 
> Diffs
> -
> 
>   build.gradle 16facbbf4dff378c561461786ff186bd9eed 
>   gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
>   samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
> 5d066c5867e9df9e94e60bde825dedf10703b399 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Scan.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/window/WindowOutput.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/join/Join.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/window/SessionWindow.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/window/Trigger.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/window/Window.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorBaseImpl.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/impl/Pipeline.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/TestScanner.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/data/TestDataStream.java
>  PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/AssembleCallGraphTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/BroadcastOperatorTask.java 
> PRE-CREATION 
>   samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorTasks.java
>  PRE-CREA

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-09-12 Thread Yi Pan (Data Infrastructure)

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/
---

(Updated Sept. 12, 2016, 5:53 p.m.)


Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.


Changes
---

High-lighted changes in this revision:
- change the type parameter for MessageStream to just use Message
- added MessageStreams to group all static methods to create input system 
MessageStream
- added Windows to group all static methods to create Window functions (first 
implementation is SessionWindow)
- added Triggers to allow a builder-style of window trigger programming
- moved all Operator classes into operator.api.internal.Operators, s.t. they 
are not directly exposed in operator.api package. Those classes are only used 
by implementation classes
- updated all test examples w/ the updated window and trigger programming APIs
- changed Pipeline to ChainedOperators (to separate from Jake's Pipeline class 
and Navina's work on StreamProcessor)


Bugs: SAMZA-914
https://issues.apache.org/jira/browse/SAMZA-914


Repository: samza


Description
---

SAMZA-914: initial draft of operator programming API. Design doc attached to 
SAMZA-914: 
https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf


Diffs (updated)
-

  build.gradle 16facbbf4dff378c561461786ff186bd9eed 
  gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
  samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
5d066c5867e9df9e94e60bde825dedf10703b399 
  samza-operator/src/main/java/org/apache/samza/operators/api/Join.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/MessageStreams.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/Triggers.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/WindowState.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/Windows.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/data/Message.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/data/WindowOutput.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/internal/Operators.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/internal/Trigger.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/internal/Window.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/ChainedOperators.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorBaseImpl.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/ProcessorContext.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/task/StreamOperatorAdaptorTask.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java 
PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/api/data/TestMessageStream.java
 PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/AssembleCallGraphTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/BroadcastOperatorTask.java 
PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/task/InputAvroSystemMessage.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java 
PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorTasks.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/WindowOperatorTask.java 
PRE-CREATION 
  
samza-sql-calcite/src/test/java/org/apache/samza/sql/calcite/schema/TestAvroSchemaConverter.java
 PRE-CREATION 
  samza-sql-core/README.md PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Data.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/EntityName.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Relation.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Schema.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Stream.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Table.java 
PRE-CREATION 
  samza

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-09-12 Thread Yi Pan (Data Infrastructure)


> On Sept. 1, 2016, 5:58 p.m., Jake Maes wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java,
> >  line 47
> > 
> >
> > Does MessageStream need to have the key type parameter?
> > 
> > It seems like it would be more flexible to have ```MessageStream``` 
> > where M could be a simple type like ```String``` in the case that you don't 
> > care about the keys, or ```Message``` for the classic case where you 
> > do care. 
> > 
> > We could start by only implementing the latter case, but at least 
> > MessageStream would be capable of the former case in the future.
> > 
> > It seems that doing this would also eliminate the need for separate 
> > map(), flatMap(), filter(), etc. methods. A single apply() method could 
> > handle each of these cases. To me, this reads better. 
> > ```
> > MessageStream.filter(new MyCustomFilter())
> > ```
> > becomes
> > ```
> > MessageStram.apply(new MyCustomFilter())
> > ```
> > 
> > The former doesn't read like English. Could be slightly better if the 
> > name was "filterWith" but apply() still feels best.
> 
> Yi Pan (Data Infrastructure) wrote:
> Let me think about it. I have tried either ways and there are needs to 
> using both K, M as parameters to MessageStream, mainly for window and join 
> functions, which needs to know the type of the key of input and output 
> messages in aggregate and join functions and stores. If MessageStream only 
> takes Message as the type parameter, there is no way to find out the key type 
> in build time and use that to define the type parameters for window and join 
> functions. Let's discuss in person tomorrow.

Made the change s.t. MessageStream is back to the format of MessageStream.


- Yi


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review147578
---


On Aug. 26, 2016, 8:43 p.m., Yi Pan (Data Infrastructure) wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/47835/
> ---
> 
> (Updated Aug. 26, 2016, 8:43 p.m.)
> 
> 
> Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
> Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.
> 
> 
> Bugs: SAMZA-914
> https://issues.apache.org/jira/browse/SAMZA-914
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> SAMZA-914: initial draft of operator programming API. Design doc attached to 
> SAMZA-914: 
> https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf
> 
> 
> Diffs
> -
> 
>   build.gradle 16facbbf4dff378c561461786ff186bd9eed 
>   gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
>   samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
> 5d066c5867e9df9e94e60bde825dedf10703b399 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Scan.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/window/WindowOutput.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/join/Join.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/window/SessionWindow.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/window/Trigger.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/window/Window.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorBaseImpl.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/impl/Pipeline.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/TestScanner.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/data/TestDataStream.java
>  PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/AssembleCallGraphTask.java 
> PRE-CREATION 
>   

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-09-02 Thread Yi Pan (Data Infrastructure)


> On Aug. 31, 2016, 2:18 a.m., Jake Maes wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java,
> >  line 147
> > 
> >
> > I think it would be useful to call out why SinkFunction is the only one 
> > that needs a MessageCollector and TaskCoordinator. 
> > 
> > I believe the reason for TaskCoordinator is because Sink is terminal, 
> > so that is the only place you need to commit(). I'm not sure about the 
> > MessageCollector though. It seems like it should be consistent with 
> > StreamOperator. Either both should have collectors or neither... unless I'm 
> > missing something.

sink needs the MessageCollector since it is terminal and needs to send the 
messages to Kafka topic. StreamOperator are generating in-memory outputs only, 
hence, no need for MessageCollector


> On Aug. 31, 2016, 2:18 a.m., Jake Maes wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java,
> >  line 152
> > 
> >
> > It's odd that the Java Function class is used for the 1 parameter case 
> > but we define a completely independent interface for the multi-parameter 
> > case. 
> > 
> > Maybe there should be a new @FunctionalInterface for this case.
> > 
> > http://stackoverflow.com/questions/27872387/can-a-java-lambda-have-more-than-1-parameter

Thanks for the suggestion. Using @FunctionalInterface does make the 
SinkFunction more generic.


> On Aug. 31, 2016, 2:18 a.m., Jake Maes wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java,
> >  line 385
> > 
> >
> > Should this have generics so users can have specific types at the very 
> > beginning or is that not possible until at least one user-defined op has 
> > processed and declared the types?

The issue is that the IncomingMessageEnvelope is not typed. Hence, it is not 
possible to figure out the type of the key and value in the incoming message in 
the framework before creating the SystemMessageStream object and pass it to the 
user. The development of Data and Schema classes are targeted to build an 
abstract data representation and access layer s.t. in the process, serde is 
hidden from the user. More detailed discussion in SAMZA-848, SAMZA-842, and 
SAMZA-429. Ideally, the SystemMessageStream should come w/ Message, via Data, we can access the fields in K and M w/o the 
need to know the serde. However, w/ the current implementation, that approach 
also has drawbacks: a) instead of having a indirect dependency on avro via 
serde implementation class, we are directly depending on avro, which brings in 
dependency management issue w/ serde implementation; b) for each serde, now we 
need to have an implementation of Data to wrap on top of it. I started
  to think that this may actually be better to be incorporated into serde class 
interface APIs, instead of the current intermediate approach. For now, I am not 
going to use this class and will always assume that incoming message is 
Message and need user to convert it to known deserialized class 
(as we do in user code today).


> On Aug. 31, 2016, 2:18 a.m., Jake Maes wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/Scan.java, line 
> > 28
> > 
> >
> > Naming:
> > Maybe it's just me and maybe I'm not seeing the big picture of use 
> > cases, but I still don't find this class name intuitive. Scan is a verb 
> > that makes some sense if the input is a table but I've never heard someone 
> > describe a "scan" of a stream. Further, to me, scan is a "read" operation 
> > and it would be unintuitive for it to have side effects like extracting 
> > keys and timestamps.   
> > 
> > Also, it seems like this will only be used at the entry points of the 
> > DAG. If so, it should be the antithesis of "sink" so I'd call it "source"
> > Source.createWithExtractors(te, ke)
> > Source.createWithKeyExtractor(ke)
> > Source.createWithTimestampExtractor(te)
> > 
> > Alternatively, if this could be used in the middle of the DAG, it could 
> > be 
> > MessageStream.createWithExtractors(te, ke)
> > MessageStream.createWithKeyExtractor(ke)
> > MessageStream.createWithTimestampExtractor(te)
> > 
> > Neither of the above are verbs so here are some other options:
> > Preprocess
> > Normalize
> > Decorate
> > Annotate
> > Extract
> > Enrich (or any of its synonyms)
> > 
> > I know they may be less concise, but they all feel clearer to me. 
> > Normalize and Enrich are my current favorites.

Chris had a similar comment and I have removed this Scan class completely from 
the API. Thanks for point it out as wel

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-09-02 Thread Yi Pan (Data Infrastructure)


> On Sept. 1, 2016, 5:58 p.m., Jake Maes wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java,
> >  line 47
> > 
> >
> > Does MessageStream need to have the key type parameter?
> > 
> > It seems like it would be more flexible to have ```MessageStream``` 
> > where M could be a simple type like ```String``` in the case that you don't 
> > care about the keys, or ```Message``` for the classic case where you 
> > do care. 
> > 
> > We could start by only implementing the latter case, but at least 
> > MessageStream would be capable of the former case in the future.
> > 
> > It seems that doing this would also eliminate the need for separate 
> > map(), flatMap(), filter(), etc. methods. A single apply() method could 
> > handle each of these cases. To me, this reads better. 
> > ```
> > MessageStream.filter(new MyCustomFilter())
> > ```
> > becomes
> > ```
> > MessageStram.apply(new MyCustomFilter())
> > ```
> > 
> > The former doesn't read like English. Could be slightly better if the 
> > name was "filterWith" but apply() still feels best.

Let me think about it. I have tried either ways and there are needs to using 
both K, M as parameters to MessageStream, mainly for window and join functions, 
which needs to know the type of the key of input and output messages in 
aggregate and join functions and stores. If MessageStream only takes Message as 
the type parameter, there is no way to find out the key type in build time and 
use that to define the type parameters for window and join functions. Let's 
discuss in person tomorrow.


- Yi


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review147578
---


On Aug. 26, 2016, 8:43 p.m., Yi Pan (Data Infrastructure) wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/47835/
> ---
> 
> (Updated Aug. 26, 2016, 8:43 p.m.)
> 
> 
> Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
> Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.
> 
> 
> Bugs: SAMZA-914
> https://issues.apache.org/jira/browse/SAMZA-914
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> SAMZA-914: initial draft of operator programming API. Design doc attached to 
> SAMZA-914: 
> https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf
> 
> 
> Diffs
> -
> 
>   build.gradle 16facbbf4dff378c561461786ff186bd9eed 
>   gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
>   samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
> 5d066c5867e9df9e94e60bde825dedf10703b399 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Scan.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/window/WindowOutput.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/join/Join.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/window/SessionWindow.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/window/Trigger.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/window/Window.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorBaseImpl.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/impl/Pipeline.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/TestScanner.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/data/TestDataStream.java
>  PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/AssembleCallGraphTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/BroadcastOperatorTask.java 
> PRE-CREATION 
>   samza-operator/src/test/java/org/

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-09-02 Thread Yi Pan (Data Infrastructure)


> On Sept. 1, 2016, 9:25 p.m., Jake Maes wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/Scan.java, 
> > lines 36-41
> > 
> >
> > Tuple/Message
> > 
> > Also, I don't see anything called "sourceTuple"or "sourceMessage" that 
> > this doc is referring to.

Thanks for catching that. Fixed (as it is gone now).


> On Sept. 1, 2016, 9:25 p.m., Jake Maes wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/data/window/WindowOutput.java,
> >  line 39
> > 
> >
> > Can this be called "setMessage"? 
> > I think that's more accurate to what's happening in the body.
> > 
> > Better yet, if this class were immutable, the method wouldn't be needed 
> > at all. Naming problem, solved!

I have made it immutable, as Chris suggest. Thanks for catching it!


> On Sept. 1, 2016, 9:25 p.m., Jake Maes wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java,
> >  line 47
> > 
> >
> > tuple/message

Yeah, docs need to be thoroughly cleaned up due to the refactoring from Tuple 
==> Message. I will fix them.


> On Sept. 1, 2016, 9:25 p.m., Jake Maes wrote:
> > samza-operator/src/test/java/org/apache/samza/task/AssembleCallGraphTask.java,
> >  lines 118-133
> > 
> >
> > This is a matter of personal taste, but I feel this code would be 
> > easier to parse and understand if the lambdas were predefined functions. 
> > 
> > For me lambdas are for the super simple cases. 
> > 
> > If the goal of this is to actually test the lambdas, then ignore this 
> > feedback.

Sure. I will make the complex lambdas as predefined functions. I assume that 
you mainly refer to sink()?


- Yi


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review147607
---


On Aug. 26, 2016, 8:43 p.m., Yi Pan (Data Infrastructure) wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/47835/
> ---
> 
> (Updated Aug. 26, 2016, 8:43 p.m.)
> 
> 
> Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
> Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.
> 
> 
> Bugs: SAMZA-914
> https://issues.apache.org/jira/browse/SAMZA-914
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> SAMZA-914: initial draft of operator programming API. Design doc attached to 
> SAMZA-914: 
> https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf
> 
> 
> Diffs
> -
> 
>   build.gradle 16facbbf4dff378c561461786ff186bd9eed 
>   gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
>   samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
> 5d066c5867e9df9e94e60bde825dedf10703b399 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Scan.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/window/WindowOutput.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/join/Join.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/window/SessionWindow.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/window/Trigger.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/window/Window.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorBaseImpl.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/impl/Pipeline.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/TestScanner.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/data/Test

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-09-01 Thread Jake Maes

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review147607
---



A few more comments. 

This is all for now.


samza-operator/src/main/java/org/apache/samza/operators/api/Scan.java (lines 36 
- 41)


Tuple/Message

Also, I don't see anything called "sourceTuple"or "sourceMessage" that this 
doc is referring to.



samza-operator/src/main/java/org/apache/samza/operators/api/data/window/WindowOutput.java
 (line 39)


Can this be called "setMessage"? 
I think that's more accurate to what's happening in the body.

Better yet, if this class were immutable, the method wouldn't be needed at 
all. Naming problem, solved!



samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
 (line 47)


tuple/message



samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java
 (lines 38 - 39)


tuple/message



samza-operator/src/test/java/org/apache/samza/task/AssembleCallGraphTask.java 
(lines 118 - 133)


This is a matter of personal taste, but I feel this code would be easier to 
parse and understand if the lambdas were predefined functions. 

For me lambdas are for the super simple cases. 

If the goal of this is to actually test the lambdas, then ignore this 
feedback.


- Jake Maes


On Aug. 26, 2016, 8:43 p.m., Yi Pan (Data Infrastructure) wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/47835/
> ---
> 
> (Updated Aug. 26, 2016, 8:43 p.m.)
> 
> 
> Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
> Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.
> 
> 
> Bugs: SAMZA-914
> https://issues.apache.org/jira/browse/SAMZA-914
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> SAMZA-914: initial draft of operator programming API. Design doc attached to 
> SAMZA-914: 
> https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf
> 
> 
> Diffs
> -
> 
>   build.gradle 16facbbf4dff378c561461786ff186bd9eed 
>   gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
>   samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
> 5d066c5867e9df9e94e60bde825dedf10703b399 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Scan.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/window/WindowOutput.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/join/Join.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/window/SessionWindow.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/window/Trigger.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/window/Window.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorBaseImpl.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/impl/Pipeline.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/TestScanner.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/data/TestDataStream.java
>  PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/AssembleCallGraphTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/BroadcastOperatorTask.java 
> PRE-CREATION 
>   samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorTasks.java
>  PRE-CREATION 
>   samza-operator/src/test/java/org/apache/samza/task/WindowOperatorTask.java 
> PRE-CR

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-09-01 Thread Jake Maes

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review147578
---




samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java 
(line 47)


Does MessageStream need to have the key type parameter?

It seems like it would be more flexible to have ```MessageStream``` 
where M could be a simple type like ```String``` in the case that you don't 
care about the keys, or ```Message``` for the classic case where you do 
care. 

We could start by only implementing the latter case, but at least 
MessageStream would be capable of the former case in the future.

It seems that doing this would also eliminate the need for separate map(), 
flatMap(), filter(), etc. methods. A single apply() method could handle each of 
these cases. To me, this reads better. 
```
MessageStream.filter(new MyCustomFilter())
```
becomes
```
MessageStram.apply(new MyCustomFilter())
```

The former doesn't read like English. Could be slightly better if the name 
was "filterWith" but apply() still feels best.


- Jake Maes


On Aug. 26, 2016, 8:43 p.m., Yi Pan (Data Infrastructure) wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/47835/
> ---
> 
> (Updated Aug. 26, 2016, 8:43 p.m.)
> 
> 
> Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
> Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.
> 
> 
> Bugs: SAMZA-914
> https://issues.apache.org/jira/browse/SAMZA-914
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> SAMZA-914: initial draft of operator programming API. Design doc attached to 
> SAMZA-914: 
> https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf
> 
> 
> Diffs
> -
> 
>   build.gradle 16facbbf4dff378c561461786ff186bd9eed 
>   gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
>   samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
> 5d066c5867e9df9e94e60bde825dedf10703b399 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Scan.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/window/WindowOutput.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/join/Join.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/window/SessionWindow.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/window/Trigger.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/window/Window.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorBaseImpl.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/impl/Pipeline.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/TestScanner.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/data/TestDataStream.java
>  PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/AssembleCallGraphTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/BroadcastOperatorTask.java 
> PRE-CREATION 
>   samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorTasks.java
>  PRE-CREATION 
>   samza-operator/src/test/java/org/apache/samza/task/WindowOperatorTask.java 
> PRE-CREATION 
>   
> samza-sql-calcite/src/test/java/org/apache/samza/sql/calcite/schema/TestAvroSchemaConverter.java
>  PRE-CREATION 
>   samza-sql-core/README.md PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Data.java 
> PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/samza/sql/api/data/EntityName.java 
> PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/samza/sql/api/data/

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-08-30 Thread Jake Maes

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review147209
---



Mostly minor feedback like naming issues so far. 
I hopped around a bit, but my thorough review will pick up at WindowOutput 
(which I'm currently trying to understand why it's in the API)


build.gradle (line 349)


It would be nice to NOT have an avro dependency or constrain it to a 
testCompile.



samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java 
(line 52)


Naming:
When I read "*System*MessageStream", my first interpretation is that it 
represents a stream of system messages. e.g. for coordination, the common case 
for broadcast streams.

After browsing the code, I think a better name for this is "SourceStream" 
or "InputStream"

Also, it doesn't seem to be adding much value. The private fields have no 
accessors. Is this placeholding for something more significant later?



samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java 
(line 80)


Minor: 
I saw K2 and M2 and found myself asking "where are K1 and M1, and do I need 
them?" I see now that the goal was to avoid "hiding" K and M from 
MessageStream, but are there any better options?

I first thought OK and OM for output key and output message, but then it 
seems like each operator is getting ready to meditate. :-)

Maybe?
OperatorBase
T=type

Same for each of the Operators



samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java 
(line 84)


Naming:
output D Stream. "D" stands for data?
Is it needed?



samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java 
(line 147)


I think it would be useful to call out why SinkFunction is the only one 
that needs a MessageCollector and TaskCoordinator. 

I believe the reason for TaskCoordinator is because Sink is terminal, so 
that is the only place you need to commit(). I'm not sure about the 
MessageCollector though. It seems like it should be consistent with 
StreamOperator. Either both should have collectors or neither... unless I'm 
missing something.



samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java 
(line 152)


It's odd that the Java Function class is used for the 1 parameter case but 
we define a completely independent interface for the multi-parameter case. 

Maybe there should be a new @FunctionalInterface for this case.

http://stackoverflow.com/questions/27872387/can-a-java-lambda-have-more-than-1-parameter



samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java 
(line 385)


Should this have generics so users can have specific types at the very 
beginning or is that not possible until at least one user-defined op has 
processed and declared the types?



samza-operator/src/main/java/org/apache/samza/operators/api/Scan.java (line 28)


Naming:
Maybe it's just me and maybe I'm not seeing the big picture of use cases, 
but I still don't find this class name intuitive. Scan is a verb that makes 
some sense if the input is a table but I've never heard someone describe a 
"scan" of a stream. Further, to me, scan is a "read" operation and it would be 
unintuitive for it to have side effects like extracting keys and timestamps.   

Also, it seems like this will only be used at the entry points of the DAG. 
If so, it should be the antithesis of "sink" so I'd call it "source"
Source.createWithExtractors(te, ke)
Source.createWithKeyExtractor(ke)
Source.createWithTimestampExtractor(te)

Alternatively, if this could be used in the middle of the DAG, it could be 
MessageStream.createWithExtractors(te, ke)
MessageStream.createWithKeyExtractor(ke)
MessageStream.createWithTimestampExtractor(te)

Neither of the above are verbs so here are some other options:
Preprocess
Normalize
Decorate
Annotate
Extract
Enrich (or any of its synonyms)

I know they may be less concise, but they all feel clearer to me. Normalize 
and Enrich are my current favorites.



samza-sql-core/src/main/java/org/apache/samza/sql/data/avro/AvroData.java 
(lines 51 - 63)


I know this isn't part of the current patch but this smells funky. All the 
@Overrides are Unsupported..., and then there are a 

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-08-29 Thread Yi Pan (Data Infrastructure)


> On Aug. 26, 2016, 4:01 p.m., Chris Pettitt wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java,
> >  line 52
> > 
> >
> > Minor: final after access modifier for consistency with the rest of the 
> > code.
> > 
> > ---
> > 
> > If this is intended to be public, I would make it top level (same for 
> > most other public nested classes). If you want to have a factory method for 
> > it, something like java's Collections (e.g. MessageStreams) is a nice way 
> > to group these.

I have moved it to MessageStreams. Thanks for the suggestion!


> On Aug. 26, 2016, 4:01 p.m., Chris Pettitt wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java,
> >  line 80
> > 
> >
> > Package private top level class?
> > 
> > ---
> > 
> > This is very much a taste thing, but I've found myself shying away from 
> > abstract base classes in favor of interfaces over the years. A class like 
> > this doesn't provide a heck of a lot of value (it's a single field holder). 
> > It could be reframed as an interface (with getOutputStream). One of the 
> > nice things that fall out of that is that when someone reviews an Operator 
> > it is very clear where output stream comes from (it's initialized in the 
> > constructor and accessed via the getter).

Make sense. The one issue is that for sink(), there is no nextStream, if we 
modify the sink method to return void. We can return null here. I am hoping to 
keep all operator classes at least have one common root s.t. it would be much 
easier to make a list of subscribers to a MessageStream typed.


> On Aug. 26, 2016, 4:01 p.m., Chris Pettitt wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java,
> >  lines 212-217
> > 
> >
> > Entry feels a little odd here. The reason is that we already 
> > know how to map Message to SK. What would happen, for example, if 
> > Message mapped to SK0 and then a call to stateCreator with 
> > Message produced Entry? Instead, it seems that we should 
> > just take SS in and produce SS out. You could maybe argue that SK should 
> > come in on update, e.g. if it was an expensive function, but I suspect you 
> > don't want the function to be expensive anyway as this is going to be 
> > invoked for each message.
> 
> Yi Pan (Data Infrastructure) wrote:
> Good point. I will make the change s.t. it will be more consistent 
> regarding to which key in the state store a certain input message would be 
> applied to.

This StoreFunctions methods are designed w/ the thought of extension to 1 input 
messages maps to multiple state store entries in mind (i.e. sliding window, 
hopping window, non-unique-key based join). Since it is not clear how we are 
implementing these state stores for various different operators yet, I am 
reducing the scope to only target for 1 input message to 1 state store entry 
now. I will re-visit this when come back to implement more general state store 
functions. It should not affect the user APIs since the state store APIs are 
not yet exposed to the user.


- Yi


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review146969
---


On Aug. 26, 2016, 8:43 p.m., Yi Pan (Data Infrastructure) wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/47835/
> ---
> 
> (Updated Aug. 26, 2016, 8:43 p.m.)
> 
> 
> Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
> Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.
> 
> 
> Bugs: SAMZA-914
> https://issues.apache.org/jira/browse/SAMZA-914
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> SAMZA-914: initial draft of operator programming API. Design doc attached to 
> SAMZA-914: 
> https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf
> 
> 
> Diffs
> -
> 
>   build.gradle 16facbbf4dff378c561461786ff186bd9eed 
>   gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
>   samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
> 5d066c5867e9df9e94e60bde825dedf10703b399 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Scan.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/window/Wi

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-08-29 Thread Yi Pan (Data Infrastructure)


> On Aug. 26, 2016, 5:27 p.m., Chris Pettitt wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/Scan.java, line 
> > 32
> > 
> >
> > I wonder why the user doesn't just write the helper to extract 
> > timestamp, message key, etc. and then use it where they need it? The most 
> > obvious reason (maybe you had another in mind) would be that we can do this 
> > in one place and then propagate it throughout the pipeline. I think this 
> > would break e.g. in places where map is used and the output of map does not 
> > propagate the timestamp and message key explicitly. It also doesn't compose 
> > well with other output types (e.g. WindowOutput).

Yes, you are right in the sense to "scan once and used it later". I had two use 
cases in mind when I thought Scan would be useful: a) use Scan to associate 
event-time and message key to the incoming messages s.t. window can use these 
information in the downstream; b) use Scan to convert a static "Table" to 
stream w/ event-time and message-keys in the future. I realized now that w/ 
Window.windowKey() method, motivation a) is covered and b) can be saved later 
when we implement the table-to-stream scan. I will remove it now. Thanks for 
the suggestion.


> On Aug. 26, 2016, 5:27 p.m., Chris Pettitt wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/data/window/WindowOutput.java,
> >  line 30
> > 
> >
> > It would be a really nice property to keep this immutable is possible. 
> > What are the down sides?

Good point. I will change it to immutable. The initial idea is to potentially 
use this class as the object stored in the window state store as well, which 
can use this update() method. I will remove it since it is not clear yet what 
would be needed in actual window state store implementation.


> On Aug. 26, 2016, 5:27 p.m., Chris Pettitt wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/join/Join.java, 
> > lines 58-59
> > 
> >
> > The asymmetry of PartialJoin is starting to make more sense. However, 
> > given its rather specialized use for this case I wonder if we shouldn't 
> > either inline it or make it package private.
> > 
> > Also, I still wonder about treating the second input of the join as a 
> > stream versus consuming from both streams and potentially building up state 
> > for the join. Given that this is split up across a few classes, I wonder 
> > are we only mapping on the key that is coming infor the current message or 
> > are we remapping for every message?

Are you proposing to move Join class into MessageStream as an embedded class? 
Or making MessageStream.join() package private? I think that it makes more 
sense to make MessageStream.join() pacakge private.

So, as for the join state, one potential alternative is to implement a 
CoGroupByKey equivalent from Google Dataflow, in which all messages from both 
streams having the same key are grouped into a collection, and compute the join 
output from the whole collection when the PCollection transformation is 
considered done. Is it the model that you are considering? I felt that this is 
more like a batch mode and makes the state store a) record schema a 
hybrid/mixed one, b) per key record size to be big, which may cause complexity 
in our RocksDB KV-store implementation w/ (possibly) changelog.

I don't fully understand the part that you referred to as "remapping for every 
message". Can we sync up in our Mon meeting?


- Yi


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review146983
---


On Aug. 26, 2016, 8:43 p.m., Yi Pan (Data Infrastructure) wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/47835/
> ---
> 
> (Updated Aug. 26, 2016, 8:43 p.m.)
> 
> 
> Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
> Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.
> 
> 
> Bugs: SAMZA-914
> https://issues.apache.org/jira/browse/SAMZA-914
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> SAMZA-914: initial draft of operator programming API. Design doc attached to 
> SAMZA-914: 
> https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf
> 
> 
> Diffs
> -
> 
>   build.gradle 16facbbf4dff378c561461786ff186bd9eed 
>   gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
>   samza-api/src/test/java/org/apache/sam

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-08-26 Thread Yi Pan (Data Infrastructure)


> On Aug. 26, 2016, 7:34 p.m., Chris Pettitt wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/window/Trigger.java,
> >  line 28
> > 
> >
> > Definitely not digging the mutable state here.
> > 
> > Trigger a = Trigger.of(...);
> > Trigger b = Trigger.of(...);
> > Trigger c = Trigger.of(...);
> > 
> > Trigger ab = a.and(b);
> > Trigger ac = a.and(c);
> > 
> > Now both ab and ac are really a and b and c. Imagine if this weren't 
> > all in one function!
> > 
> > It appears that Trigger may not actually be necessary. Essentially this 
> > class is just a set of helpers for building predicates. I could imagine 
> > something like this:
> > 
> > ```
> > public static class PredicateHelper {
> > public static  Function and(Function 
> > lhs, Function rhs) {
> > return new Function() {
> > @Override
> > public Boolean apply(S s) {
> > return lhs.apply(s) && rhs.apply(s);
> > }
> > };
> > }
> > }
> > ```
> > 
> > Main point though is to use immutability.

You are completely right that this is a way to build predicates. I will try 
this out w/ the new Trigger API updates.


> On Aug. 26, 2016, 7:34 p.m., Chris Pettitt wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/window/Window.java,
> >  line 122
> > 
> >
> > Minor: we should be consistent with camel case.

Yep, fixed in the updated patch.


> On Aug. 26, 2016, 7:34 p.m., Chris Pettitt wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/window/Window.java,
> >  line 41
> > 
> >
> > Last time I'll beat on this dead horse. Definitely favor immutability 
> > where you can. Left to the reader to determine whether that is desirable.

Are you referring to all the member variables being final when mentioning 
"immutability"? The problem I have w/ is that the user will now need to specify 
all variables in constructor. W/ all possible combination of user choices in 
whether include/use a certain function or not, the versions of constructor 
would explode. For example, when a user choose to specify watermark or 
lateArrival optionally, there will need to be different versions of constructor 
for this class to support the immutability. Also, there has been a use case in 
Calcite integration we had before, that the query planner/optimizer does not 
know the actual parameters for the operator yet while creating the operator 
instance. The desire was to use a builder pattern to gradually build-up the 
operator. Let's sync up offline.


> On Aug. 26, 2016, 7:34 p.m., Chris Pettitt wrote:
> > samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java, 
> > lines 45-59
> > 
> >
> > If this is all the user really needs to implement could we do this as 
> > an interface instead?

Let me try this solution. One question that I have is: there is some logic that 
I wanted to enforce regarding to when to call the user implemented initOnce() 
and initPipeline() methods in the StreamOperatorTask.init(). How do I enforece 
it if I convert StreamOperatorTask to an interface class?


> On Aug. 26, 2016, 7:34 p.m., Chris Pettitt wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/window/Window.java,
> >  line 131
> > 
> >
> > If the only reason to make this abstract is the function can we just 
> > take it as an arg and about class hierarchy?

I can remove it for now, since there is only one implementation of Window 
class. This might be needed later in the window implementation classes, when we 
have more windows implemented. I do have one thing that can't find a solution 
for yet: these functions are actually only accessed within the implementation 
classes in Samza, should not be accessed by the programmer. How do I hide them 
while still provide external access to those method in 
org.apache.samza.operator.impl package? The only potential solution I can think 
of is to fold these methods into package private and expose to implementation 
classes via WindowOperator inside MessageStream. However, I will have to move 
Window related classes to the same package path w/ MessageStream, which makes 
the source file structure less modular. I would be appreciate if you have any 
suggestion on this! Thanks!


- Yi


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review147014
---

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-08-26 Thread Yi Pan (Data Infrastructure)


> On Aug. 26, 2016, 4:01 p.m., Chris Pettitt wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java,
> >  lines 152-154
> > 
> >
> > It's not obvious why a sink function would produce output. Even a tee 
> > function should be pass-through. Should our return type be void here?

Make sense. The original idea is to implement a Tee-like operator and make the 
output to be used optionally. It does create confusion in semantics. Let me 
clean it up.


> On Aug. 26, 2016, 4:01 p.m., Chris Pettitt wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java,
> >  line 161
> > 
> >
> > If we're going to pass through the data this might better be called a 
> > TeeOperator. I'd expect a sink to produce no output.

Make sense to me. Thanks!


> On Aug. 26, 2016, 4:01 p.m., Chris Pettitt wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java,
> >  line 197
> > 
> >
> > It is not clear from the documentation if you're allowed to produce no 
> > new state for a given incoming message. Given the way the interface is 
> > defined, I could see doing this in a slightly inconsistent way: for create 
> > you pass back a null SS, but for update you return back the input value 
> > (identity function). I wonder if we couldn't just have an update function 
> > that could take a null input. A potential side benefit would be that you 
> > could now delete state for a key (if you needed to) by producing a null 
> > output from update.

Thanks for point it out. I debated myself on whether we want two functions or a 
combined update function as you suggest. The reason I left the two functions 
here is that I am not sure whether the state store programming APIs that we 
expose to the user would need to differentiate those two cases. But it makes 
sense to converge to a single function even the state store programming APIs 
have multiple distinct functions to perform insert/update/delete in the state 
store. BTW, the state store programming APIs are intentionally left out in this 
patch and need to be worked on later.


> On Aug. 26, 2016, 4:01 p.m., Chris Pettitt wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java,
> >  line 197
> > 
> >
> > It is not clear from the documentation if you're allowed to produce no 
> > new state for a given incoming message. Given the way the interface is 
> > defined, I could see doing this in a slightly inconsistent way: for create 
> > you pass back a null SS, but for update you return back the input value 
> > (identity function). I wonder if we couldn't just have an update function 
> > that could take a null input. A potential side benefit would be that you 
> > could now delete state for a key (if you needed to) by producing a null 
> > output from update.

Dup as the above. Please see my replies there.


> On Aug. 26, 2016, 4:01 p.m., Chris Pettitt wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java,
> >  lines 212-217
> > 
> >
> > Entry feels a little odd here. The reason is that we already 
> > know how to map Message to SK. What would happen, for example, if 
> > Message mapped to SK0 and then a call to stateCreator with 
> > Message produced Entry? Instead, it seems that we should 
> > just take SS in and produce SS out. You could maybe argue that SK should 
> > come in on update, e.g. if it was an expensive function, but I suspect you 
> > don't want the function to be expensive anyway as this is going to be 
> > invoked for each message.

Good point. I will make the change s.t. it will be more consistent regarding to 
which key in the state store a certain input message would be applied to.


> On Aug. 26, 2016, 4:01 p.m., Chris Pettitt wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java,
> >  line 246
> > 
> >
> > It would probably be clearer to say "generate an output w/ output type 
> > {@code Message}. It's not immediately obvious what type R is here. 
> > I would also touch on the fact that (IIUC) the output is going to be a 
> > Collection> but that this is going to be flattened down to 
> > a regular stream of Message.

Good point. Done.


> On Aug. 26, 2016, 4:01 p.m., Chris Pettitt wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java,
> >  line 300
> > 
> >
> > Would it be possi

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-08-26 Thread Yi Pan (Data Infrastructure)

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/
---

(Updated Aug. 26, 2016, 8:43 p.m.)


Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.


Changes
---

This is an update trying to make Trigger APIs easier for user to specify. Have 
not addressed Chris' feedbacks in revision 12 yet. Will work on that now.

Thanks!


Bugs: SAMZA-914
https://issues.apache.org/jira/browse/SAMZA-914


Repository: samza


Description
---

SAMZA-914: initial draft of operator programming API. Design doc attached to 
SAMZA-914: 
https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf


Diffs (updated)
-

  build.gradle 16facbbf4dff378c561461786ff186bd9eed 
  gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
  samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
5d066c5867e9df9e94e60bde825dedf10703b399 
  
samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/Scan.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/data/window/WindowOutput.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/join/Join.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/window/SessionWindow.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/window/Trigger.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/window/Window.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorBaseImpl.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/impl/Pipeline.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/operators/api/TestScanner.java 
PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/api/data/TestDataStream.java
 PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/AssembleCallGraphTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/BroadcastOperatorTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java 
PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorTasks.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/WindowOperatorTask.java 
PRE-CREATION 
  
samza-sql-calcite/src/test/java/org/apache/samza/sql/calcite/schema/TestAvroSchemaConverter.java
 PRE-CREATION 
  samza-sql-core/README.md PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Data.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/EntityName.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Relation.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Schema.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Stream.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Table.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Tuple.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/Operator.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorCallback.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorRouter.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorSpec.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SimpleOperator.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/IncomingMessageTuple.java
 PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/data/avro/AvroData.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/data/avro/AvroSchema.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlAvroSerde.java
 PRE-CREATION 
  
samz

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-08-26 Thread Chris Pettitt

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review147014
---



Only skimmed test cases. Let's review the items captured so far and then I can 
do another pass on tests.


samza-operator/src/main/java/org/apache/samza/operators/api/window/Trigger.java 
(line 28)


Definitely not digging the mutable state here.

Trigger a = Trigger.of(...);
Trigger b = Trigger.of(...);
Trigger c = Trigger.of(...);

Trigger ab = a.and(b);
Trigger ac = a.and(c);

Now both ab and ac are really a and b and c. Imagine if this weren't all in 
one function!

It appears that Trigger may not actually be necessary. Essentially this 
class is just a set of helpers for building predicates. I could imagine 
something like this:

```
public static class PredicateHelper {
public static  Function and(Function lhs, 
Function rhs) {
return new Function() {
@Override
public Boolean apply(S s) {
return lhs.apply(s) && rhs.apply(s);
}
};
}
}
```

Main point though is to use immutability.



samza-operator/src/main/java/org/apache/samza/operators/api/window/Window.java 
(line 41)


Last time I'll beat on this dead horse. Definitely favor immutability where 
you can. Left to the reader to determine whether that is desirable.



samza-operator/src/main/java/org/apache/samza/operators/api/window/Window.java 
(line 122)


Minor: we should be consistent with camel case.



samza-operator/src/main/java/org/apache/samza/operators/api/window/Window.java 
(line 131)


If the only reason to make this abstract is the function can we just take 
it as an arg and about class hierarchy?



samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java 
(lines 45 - 59)


If this is all the user really needs to implement could we do this as an 
interface instead?


- Chris Pettitt


On Aug. 23, 2016, 7:23 a.m., Yi Pan (Data Infrastructure) wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/47835/
> ---
> 
> (Updated Aug. 23, 2016, 7:23 a.m.)
> 
> 
> Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
> Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.
> 
> 
> Bugs: SAMZA-914
> https://issues.apache.org/jira/browse/SAMZA-914
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> SAMZA-914: initial draft of operator programming API. Design doc attached to 
> SAMZA-914: 
> https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf
> 
> 
> Diffs
> -
> 
>   build.gradle 16facbbf4dff378c561461786ff186bd9eed 
>   gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
>   samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
> 5d066c5867e9df9e94e60bde825dedf10703b399 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Scan.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/window/WindowOutput.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/join/Join.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/window/SessionWindow.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/window/Trigger.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/window/Window.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorBaseImpl.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/impl/Pipeline.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-08-26 Thread Chris Pettitt

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review146983
---



Next meeting. Pick this up from Trigger (watching too much Winnie the Pooh as 
my hands have written Tigger multiple times!).


samza-operator/src/main/java/org/apache/samza/operators/api/Scan.java (line 32)


I wonder why the user doesn't just write the helper to extract timestamp, 
message key, etc. and then use it where they need it? The most obvious reason 
(maybe you had another in mind) would be that we can do this in one place and 
then propagate it throughout the pipeline. I think this would break e.g. in 
places where map is used and the output of map does not propagate the timestamp 
and message key explicitly. It also doesn't compose well with other output 
types (e.g. WindowOutput).



samza-operator/src/main/java/org/apache/samza/operators/api/data/window/WindowOutput.java
 (line 30)


It would be a really nice property to keep this immutable is possible. What 
are the down sides?



samza-operator/src/main/java/org/apache/samza/operators/api/join/Join.java 
(lines 58 - 59)


The asymmetry of PartialJoin is starting to make more sense. However, given 
its rather specialized use for this case I wonder if we shouldn't either inline 
it or make it package private.

Also, I still wonder about treating the second input of the join as a 
stream versus consuming from both streams and potentially building up state for 
the join. Given that this is split up across a few classes, I wonder are we 
only mapping on the key that is coming infor the current message or are we 
remapping for every message?


- Chris Pettitt


On Aug. 23, 2016, 7:23 a.m., Yi Pan (Data Infrastructure) wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/47835/
> ---
> 
> (Updated Aug. 23, 2016, 7:23 a.m.)
> 
> 
> Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
> Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.
> 
> 
> Bugs: SAMZA-914
> https://issues.apache.org/jira/browse/SAMZA-914
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> SAMZA-914: initial draft of operator programming API. Design doc attached to 
> SAMZA-914: 
> https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf
> 
> 
> Diffs
> -
> 
>   build.gradle 16facbbf4dff378c561461786ff186bd9eed 
>   gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
>   samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
> 5d066c5867e9df9e94e60bde825dedf10703b399 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Scan.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/window/WindowOutput.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/join/Join.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/window/SessionWindow.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/window/Trigger.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/window/Window.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorBaseImpl.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/impl/Pipeline.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/TestScanner.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/data/TestDataStream.java
>  PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/AssembleCallGraphTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/task/BroadcastOperatorTask.java 
> PRE-CREATION 
>   samza-operator/src/test/java/org/apache/samza/task/J

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-08-26 Thread Chris Pettitt

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review146969
---



Breaking for lunch, but don't want to risk losing my comments. Will pick up 
from "Scan".


samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java 
(line 52)


Minor: final after access modifier for consistency with the rest of the 
code.

---

If this is intended to be public, I would make it top level (same for most 
other public nested classes). If you want to have a factory method for it, 
something like java's Collections (e.g. MessageStreams) is a nice way to group 
these.



samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java 
(line 80)


Package private top level class?

---

This is very much a taste thing, but I've found myself shying away from 
abstract base classes in favor of interfaces over the years. A class like this 
doesn't provide a heck of a lot of value (it's a single field holder). It could 
be reframed as an interface (with getOutputStream). One of the nice things that 
fall out of that is that when someone reviews an Operator it is very clear 
where output stream comes from (it's initialized in the constructor and 
accessed via the getter).



samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java 
(lines 152 - 154)


It's not obvious why a sink function would produce output. Even a tee 
function should be pass-through. Should our return type be void here?



samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java 
(line 161)


If we're going to pass through the data this might better be called a 
TeeOperator. I'd expect a sink to produce no output.



samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java 
(line 197)


It is not clear from the documentation if you're allowed to produce no new 
state for a given incoming message. Given the way the interface is defined, I 
could see doing this in a slightly inconsistent way: for create you pass back a 
null SS, but for update you return back the input value (identity function). I 
wonder if we couldn't just have an update function that could take a null 
input. A potential side benefit would be that you could now delete state for a 
key (if you needed to) by producing a null output from update.



samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java 
(line 197)


It is not clear from the documentation if you're allowed to produce no new 
state for a given incoming message. Given the way the interface is defined, I 
could see doing this in a slightly inconsistent way: for create you pass back a 
null SS, but for update you return back the input value (identity function). I 
wonder if we couldn't just have an update function that could take a null 
input. A potential side benefit would be that you could now delete state for a 
key (if you needed to) by producing a null output from update.



samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java 
(lines 212 - 217)


Entry feels a little odd here. The reason is that we already know 
how to map Message to SK. What would happen, for example, if Message mapped to SK0 and then a call to stateCreator with Message produced 
Entry? Instead, it seems that we should just take SS in and produce 
SS out. You could maybe argue that SK should come in on update, e.g. if it was 
an expensive function, but I suspect you don't want the function to be 
expensive anyway as this is going to be invoked for each message.



samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java 
(line 246)


It would probably be clearer to say "generate an output w/ output type 
{@code Message}. It's not immediately obvious what type R is here. I 
would also touch on the fact that (IIUC) the output is going to be a 
Collection> but that this is going to be flattened down to a 
regular stream of Message.



samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java 
(line 300)


Would it be possible that users would want to specify the store name 
themselves (e.g. for simplifying configuration and for being able to inspect 
state)?



samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java 
(line 318)


Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-08-23 Thread Yi Pan (Data Infrastructure)

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/
---

(Updated Aug. 23, 2016, 7:23 a.m.)


Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.


Changes
---

Added blacklist, timeout + limit watermark trigger to ACG example.


Bugs: SAMZA-914
https://issues.apache.org/jira/browse/SAMZA-914


Repository: samza


Description
---

SAMZA-914: initial draft of operator programming API. Design doc attached to 
SAMZA-914: 
https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf


Diffs (updated)
-

  build.gradle 16facbbf4dff378c561461786ff186bd9eed 
  gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
  samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
5d066c5867e9df9e94e60bde825dedf10703b399 
  
samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/Scan.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/data/window/WindowOutput.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/join/Join.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/window/SessionWindow.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/window/Trigger.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/window/Window.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorBaseImpl.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/impl/Pipeline.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/operators/api/TestScanner.java 
PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/api/data/TestDataStream.java
 PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/AssembleCallGraphTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/BroadcastOperatorTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java 
PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorTasks.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/WindowOperatorTask.java 
PRE-CREATION 
  
samza-sql-calcite/src/test/java/org/apache/samza/sql/calcite/schema/TestAvroSchemaConverter.java
 PRE-CREATION 
  samza-sql-core/README.md PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Data.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/EntityName.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Relation.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Schema.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Stream.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Table.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Tuple.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/Operator.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorCallback.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorRouter.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorSpec.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SimpleOperator.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/IncomingMessageTuple.java
 PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/data/avro/AvroData.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/data/avro/AvroSchema.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlAvroSerde.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlAvroSerdeFactory.java
 PRE-CRE

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-08-15 Thread Yi Pan (Data Infrastructure)

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/
---

(Updated Aug. 15, 2016, 9:47 a.m.)


Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.


Changes
---

Updates:
- added ACG code example
- added timer driven method in WindowOperator (onTimer)
- modified Window function to have groupBy() and count() methods.


Bugs: SAMZA-914
https://issues.apache.org/jira/browse/SAMZA-914


Repository: samza


Description
---

SAMZA-914: initial draft of operator programming API. Design doc attached to 
SAMZA-914: 
https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf


Diffs (updated)
-

  build.gradle 16facbbf4dff378c561461786ff186bd9eed 
  gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
  samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
5d066c5867e9df9e94e60bde825dedf10703b399 
  
samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/Scan.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/data/window/WindowOutput.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/join/Join.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/window/SessionWindow.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/window/Window.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorBaseImpl.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/impl/Pipeline.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/operators/api/TestScanner.java 
PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/api/data/TestDataStream.java
 PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/AssembleCallGraphTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/BroadcastOperatorTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java 
PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorTasks.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/WindowOperatorTask.java 
PRE-CREATION 
  
samza-sql-calcite/src/test/java/org/apache/samza/sql/calcite/schema/TestAvroSchemaConverter.java
 PRE-CREATION 
  samza-sql-core/README.md PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Data.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/EntityName.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Relation.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Schema.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Stream.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Table.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Tuple.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/Operator.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorCallback.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorRouter.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorSpec.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SimpleOperator.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/IncomingMessageTuple.java
 PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/data/avro/AvroData.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/data/avro/AvroSchema.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlAvroSerde.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlAvroSerdeFactory.java
 PRE-CREATION 
  

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-08-11 Thread Yi Pan (Data Infrastructure)

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/
---

(Updated Aug. 11, 2016, 6:50 p.m.)


Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.


Changes
---

Updates:
- updated MessageStream to take explicit key and message type parameters. 
- Change join function to only take keys in the input join streams as join key. 
- Change window function to operate on input message as a groupBy function as 
default. 
- Add count as a special groupBy() method in window function.


Bugs: SAMZA-914
https://issues.apache.org/jira/browse/SAMZA-914


Repository: samza


Description
---

SAMZA-914: initial draft of operator programming API. Design doc attached to 
SAMZA-914: 
https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf


Diffs (updated)
-

  build.gradle 16facbbf4dff378c561461786ff186bd9eed 
  gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
  samza-api/src/test/java/org/apache/samza/config/TestConfig.java 
5d066c5867e9df9e94e60bde825dedf10703b399 
  
samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/Scan.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/data/window/WindowOutput.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/join/Join.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/window/SessionWindow.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/window/Window.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorBaseImpl.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/impl/Pipeline.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/operators/api/TestScanner.java 
PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/api/data/TestDataStream.java
 PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/BroadcastOperatorTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java 
PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorTasks.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/WindowOperatorTask.java 
PRE-CREATION 
  
samza-sql-calcite/src/test/java/org/apache/samza/sql/calcite/schema/TestAvroSchemaConverter.java
 PRE-CREATION 
  samza-sql-core/README.md PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Data.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/EntityName.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Relation.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Schema.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Stream.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Table.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Tuple.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/Operator.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorCallback.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorRouter.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorSpec.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SimpleOperator.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/IncomingMessageTuple.java
 PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/data/avro/AvroData.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/data/avro/AvroSchema.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlAvroSerde.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlAvroSerdeFactory.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-08-08 Thread Yi Pan (Data Infrastructure)

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/
---

(Updated Aug. 8, 2016, 4:56 p.m.)


Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.


Changes
---

Revision pass 2:
- renamed Tuple to Message and DataStream to MessageStream
- change Join.join() to static function and takes two input MessageStreams as 
input parameters
- change Join.join() for easier programming for unique-key based join
- change Window class to support counter as the first class aggregation method
- remove the mandatory Data type in most classes and only use it for Message w/ 
serialized format


Bugs: SAMZA-914
https://issues.apache.org/jira/browse/SAMZA-914


Repository: samza


Description
---

SAMZA-914: initial draft of operator programming API. Design doc attached to 
SAMZA-914: 
https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf


Diffs (updated)
-

  build.gradle 16facbbf4dff378c561461786ff186bd9eed 
  gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
  
samza-operator/src/main/java/org/apache/samza/operators/api/MessageStream.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/Scan.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/join/Join.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/window/SessionWindow.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/window/Window.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorBaseImpl.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/impl/Pipeline.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/operators/api/TestScanner.java 
PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/api/data/TestDataStream.java
 PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/BroadcastOperatorTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java 
PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorTasks.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/WindowOperatorTask.java 
PRE-CREATION 
  
samza-sql-calcite/src/test/java/org/apache/samza/sql/calcite/schema/TestAvroSchemaConverter.java
 PRE-CREATION 
  samza-sql-core/README.md PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Data.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/EntityName.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Relation.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Schema.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Stream.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Table.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Tuple.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/Operator.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorCallback.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorRouter.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorSpec.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SimpleOperator.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/IncomingMessageTuple.java
 PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/data/avro/AvroData.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/data/avro/AvroSchema.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlAvroSerde.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlAvroSerdeFactory.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlStringSerde.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/sa

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-08-02 Thread Yi Pan (Data Infrastructure)

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/
---

(Updated Aug. 2, 2016, 7:44 a.m.)


Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.


Bugs: SAMZA-914
https://issues.apache.org/jira/browse/SAMZA-914


Repository: samza


Description (updated)
---

SAMZA-914: initial draft of operator programming API. Design doc attached to 
SAMZA-914: 
https://issues.apache.org/jira/secure/attachment/12821524/SAMZA-914_%20operator%20Java%20programming%20API%20-%20Google%20Docs.pdf


Diffs (updated)
-

  build.gradle 16facbbf4dff378c561461786ff186bd9eed 
  gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
  samza-operator/src/main/java/org/apache/samza/operators/api/DataStream.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/Scan.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/join/Join.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/window/Window.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorBaseImpl.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/impl/Pipeline.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/task/DataStreamTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/operators/api/TestScanner.java 
PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/api/data/TestDataStream.java
 PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/DataStreamJoinTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/DataStreamSplitTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/DataStreamUserTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/TestDataStreamTask.java 
PRE-CREATION 
  
samza-sql-calcite/src/test/java/org/apache/samza/sql/calcite/schema/TestAvroSchemaConverter.java
 PRE-CREATION 
  samza-sql-core/README.md PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Data.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/EntityName.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Relation.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Schema.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Stream.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Table.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Tuple.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/Operator.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorCallback.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorRouter.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorSpec.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SimpleOperator.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/IncomingMessageTuple.java
 PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/data/avro/AvroData.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/data/avro/AvroSchema.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlAvroSerde.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlAvroSerdeFactory.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlStringSerde.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlStringSerdeFactory.java
 PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/data/string/StringData.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/string/StringSchema.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/NoopOperatorCallback.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorFactoryImpl.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/operato

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-07-19 Thread Yi Pan (Data Infrastructure)


> On July 18, 2016, 8:01 p.m., Chris Pettitt wrote:
> > The description says this is the initial draft implementation, but the 
> > title says initial draft for the APIs. I take it the latter is more 
> > accurate?
> > 
> > Some initial high-level thoughts:
> > 
> > 1. We should use Java 8 constructs where possible (if we've moved to Java 
> > 8). For example, Function. If not, we should probably have some Function 
> > equivalent and various type specifications for map, flatMap, etc.
> > 2. The stream operators don't seem to **do** anything - they primarily 
> > appear to hold metadata about something to be done. Are they intended only 
> > to be declarative? Are they also not intended to know directly how to build 
> > themselves, but rather you would run some graph processor (with knowledge 
> > about each operator type) to build out a real processor? This is pretty 
> > fundamental to the design. If there is a doc that covers this let me know; 
> > otherwise it would be super helpful to be on the same page about the end 
> > goal.
> > 3. If bullet 2 is correct, how would you navigate through the graph? It 
> > would seem that you would need some way to navigate `(source stream, 
> > operator)` tuples?
> > 4. Related to bullet 3, why do the operators know about their output 
> > streams? Abstractly aren't they totally independent in the sense that I 
> > could apply the same operator to multiple input streams to produce a 
> > corresponding number of output streams?
> > 5. It looks like you can compose graphs in two ways: directly using the 
> > operators or using the interfaces on DataStream. I would choose one or the 
> > other and use appropriate hiding mechanisms to expose just the API the user 
> > should be concerned with. If you go with the latter I would extract 
> > interfaces for the operators (if that is even necessary) into a public 
> > package and hide everything else in a package private namespace.
> > 6. I suspect you want to limit the ability to create custom operators (at 
> > least if the assumption about how graph walking would work in bullet 2 
> > holds), so StreamOperator's constructor probably needs to be package 
> > private.
> > 
> > I suspect some over arching docs or a bluejeans session would be very 
> > helpful in allowing me to dig deeper into this.
> 
> Yi Pan (Data Infrastructure) wrote:
> Thanks for the quick review comments! I am working on an design doc right 
> now and will share it w/ the team soon. I will take your feedbacks in the 
> design doc. We can have the discussion in our coming meeting soon.
> 
> Just some answers to your bullet points:
> 1. Sure. I will change it to use JDK constructs.
> 2. Yes. My overall thoughts on the design is: a) stream operators are 
> APIs that allows users to describe the functions to be applied on an input 
> DataStream and output to a DataStream; b) DataStream is the construct to 
> compose the DAG for operators; c) there will be implementation classes for 
> each stream operator that actually applies the user-defined processing 
> functions to the input tuples (in SAMZA-915 RB rb47994). These implementation 
> classes know how to build themselves from a descriptive stream operator, 
> which just contains the metadata of the operator. I am not sure what you 
> refer to as "graph processor", but I think that the closest thing to it in 
> the current RBs would be the Pipeline class, which connects all 
> implementation classes of stream operators when Pipeline.create() is invoked. 
> This definitely needs a design doc and I am working on it as suggest by 
> Kartik as well. Thanks for urging me on this and please let me know if you 
> have better suggestions to simplify the 
 design.
> 3. Yes. I have used reactive stream library to create the base class for 
> all oprator implementations, which I used in rb47994 to navigate through DAGs.
> 4. The reason that I associate one output DataStream to each operator 
> instance is the following. Although the descriptive Function in each 
> StreamOperator are independent of input/output DataStreams, each instance of 
> StreamOperator represent an actual transformation stage from one input 
> DataStream to one output DataStream. The special case is for Joiner, which I 
> plan to implement as a meta-class to hold the overall JoinCondition and 
> JoinFunction, but internally, implement two PartialJoinOperators that takes 
> one input, join w/ buffered tuples from the other, and send to the common 
> output R. Hence, when we translate the whole DAG in operator implementation 
> classes, each operator instance has only one input and one output. We have 
> discussed the "split" operator before and have concluded that it can be 
> implemented by applying different filters to the common output from an 
> operator. W/ this, I figure that we can use the operator only implementing 
> 1-input-1-output to construct arbitra
 ry m-input-n-output DAG nodes. Please let me know

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-07-19 Thread Yi Pan (Data Infrastructure)


> On July 18, 2016, 8:01 p.m., Chris Pettitt wrote:
> > samza-operator/src/main/java/org/apache/samza/operators/api/FlatMapper.java,
> >  line 34
> > 
> >
> > Have we moved to Java 8 yet? If so, it would be preferable to use the 
> > Java 8 constructs if possible. If not, I think this can be generalized a 
> > little more nicely as a function from T to a Collection of R instead of 
> > having a separate flat map function type.
> > 
> > This comment can be applied generally wherever we're providing 
> > "functions"

Thanks! Generalize the Function interface sounds good to me. I will make the 
corresponding modifications.


- Yi


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review142421
---


On July 19, 2016, 6:04 p.m., Yi Pan (Data Infrastructure) wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/47835/
> ---
> 
> (Updated July 19, 2016, 6:04 p.m.)
> 
> 
> Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
> Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.
> 
> 
> Bugs: SAMZA-914
> https://issues.apache.org/jira/browse/SAMZA-914
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> SAMZA-914: initial draft of operator programming API.
> 
> 
> Diffs
> -
> 
>   build.gradle 16facbbf4dff378c561461786ff186bd9eed 
>   gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
>   samza-operator/src/main/java/org/apache/samza/operators/api/FlatMapper.java 
> PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Mapper.java 
> PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Scanner.java 
> PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Sink.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/DataStream.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/join/Joiner.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/window/SessionWindow.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/window/Window.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/impl/Pipeline.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/task/DataStreamTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/TestScanner.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/data/TestDataStream.java
>  PRE-CREATION 
>   samza-operator/src/test/java/org/apache/samza/task/DataStreamJoinTask.java 
> PRE-CREATION 
>   samza-operator/src/test/java/org/apache/samza/task/DataStreamSplitTask.java 
> PRE-CREATION 
>   samza-operator/src/test/java/org/apache/samza/task/DataStreamUserTask.java 
> PRE-CREATION 
>   samza-operator/src/test/java/org/apache/samza/task/TestDataStreamTask.java 
> PRE-CREATION 
>   
> samza-sql-calcite/src/test/java/org/apache/samza/sql/calcite/schema/TestAvroSchemaConverter.java
>  PRE-CREATION 
>   samza-sql-core/README.md PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Data.java 
> PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/samza/sql/api/data/EntityName.java 
> PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Relation.java 
> PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Schema.java 
> PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Stream.java 
> PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Table.java 
> PRE-CREATION 
>   samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Tuple.java 
> PRE-CREATION 
>   
> samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/Operator.java 
> PRE-CREATION 
>   
> samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorCallback.java
>  PRE-CREATION 
>   
> samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorRouter.java
>  PRE-CREATION 
>   
> samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorSpec.java
>  PRE-CREATION 
>   
> samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SimpleOperator.java
>  PRE-CREATION 
>   
> samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java
>  PRE-CREATION 
>   
> samza-sql-core/src/main/java/org/apache/samza/sql/data/Inc

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-07-19 Thread Yi Pan (Data Infrastructure)


> On July 18, 2016, 8:01 p.m., Chris Pettitt wrote:
> > The description says this is the initial draft implementation, but the 
> > title says initial draft for the APIs. I take it the latter is more 
> > accurate?
> > 
> > Some initial high-level thoughts:
> > 
> > 1. We should use Java 8 constructs where possible (if we've moved to Java 
> > 8). For example, Function. If not, we should probably have some Function 
> > equivalent and various type specifications for map, flatMap, etc.
> > 2. The stream operators don't seem to **do** anything - they primarily 
> > appear to hold metadata about something to be done. Are they intended only 
> > to be declarative? Are they also not intended to know directly how to build 
> > themselves, but rather you would run some graph processor (with knowledge 
> > about each operator type) to build out a real processor? This is pretty 
> > fundamental to the design. If there is a doc that covers this let me know; 
> > otherwise it would be super helpful to be on the same page about the end 
> > goal.
> > 3. If bullet 2 is correct, how would you navigate through the graph? It 
> > would seem that you would need some way to navigate `(source stream, 
> > operator)` tuples?
> > 4. Related to bullet 3, why do the operators know about their output 
> > streams? Abstractly aren't they totally independent in the sense that I 
> > could apply the same operator to multiple input streams to produce a 
> > corresponding number of output streams?
> > 5. It looks like you can compose graphs in two ways: directly using the 
> > operators or using the interfaces on DataStream. I would choose one or the 
> > other and use appropriate hiding mechanisms to expose just the API the user 
> > should be concerned with. If you go with the latter I would extract 
> > interfaces for the operators (if that is even necessary) into a public 
> > package and hide everything else in a package private namespace.
> > 6. I suspect you want to limit the ability to create custom operators (at 
> > least if the assumption about how graph walking would work in bullet 2 
> > holds), so StreamOperator's constructor probably needs to be package 
> > private.
> > 
> > I suspect some over arching docs or a bluejeans session would be very 
> > helpful in allowing me to dig deeper into this.

Thanks for the quick review comments! I am working on an design doc right now 
and will share it w/ the team soon. I will take your feedbacks in the design 
doc. We can have the discussion in our coming meeting soon.

Just some answers to your bullet points:
1. Sure. I will change it to use JDK constructs.
2. Yes. My overall thoughts on the design is: a) stream operators are APIs that 
allows users to describe the functions to be applied on an input DataStream and 
output to a DataStream; b) DataStream is the construct to compose the DAG for 
operators; c) there will be implementation classes for each stream operator 
that actually applies the user-defined processing functions to the input tuples 
(in SAMZA-915 RB rb47994). These implementation classes know how to build 
themselves from a descriptive stream operator, which just contains the metadata 
of the operator. I am not sure what you refer to as "graph processor", but I 
think that the closest thing to it in the current RBs would be the Pipeline 
class, which connects all implementation classes of stream operators when 
Pipeline.create() is invoked. This definitely needs a design doc and I am 
working on it as suggest by Kartik as well. Thanks for urging me on this and 
please let me know if you have better suggestions to simplify the design
 .
3. Yes. I have used reactive stream library to create the base class for all 
oprator implementations, which I used in rb47994 to navigate through DAGs.
4. The reason that I associate one output DataStream to each operator instance 
is the following. Although the descriptive Function in each StreamOperator are 
independent of input/output DataStreams, each instance of StreamOperator 
represent an actual transformation stage from one input DataStream to one 
output DataStream. The special case is for Joiner, which I plan to implement as 
a meta-class to hold the overall JoinCondition and JoinFunction, but 
internally, implement two PartialJoinOperators that takes one input, join w/ 
buffered tuples from the other, and send to the common output R. Hence, when we 
translate the whole DAG in operator implementation classes, each operator 
instance has only one input and one output. We have discussed the "split" 
operator before and have concluded that it can be implemented by applying 
different filters to the common output from an operator. W/ this, I figure that 
we can use the operator only implementing 1-input-1-output to construct 
arbitrary m-i
 nput-n-output DAG nodes. Please let me know if you see better options.
5. I think w/ the current design, I am leaning toward using the DataStream to 
compose the DAG. Hence, that's why

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-07-19 Thread Yi Pan (Data Infrastructure)

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/
---

(Updated July 19, 2016, 6:04 p.m.)


Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.


Changes
---

Updated description


Bugs: SAMZA-914
https://issues.apache.org/jira/browse/SAMZA-914


Repository: samza


Description (updated)
---

SAMZA-914: initial draft of operator programming API.


Diffs
-

  build.gradle 16facbbf4dff378c561461786ff186bd9eed 
  gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
  samza-operator/src/main/java/org/apache/samza/operators/api/FlatMapper.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/Mapper.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/Scanner.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/Sink.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/data/DataStream.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/join/Joiner.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/window/SessionWindow.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/window/Window.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/impl/Pipeline.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/task/DataStreamTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/operators/api/TestScanner.java 
PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/api/data/TestDataStream.java
 PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/DataStreamJoinTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/DataStreamSplitTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/DataStreamUserTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/TestDataStreamTask.java 
PRE-CREATION 
  
samza-sql-calcite/src/test/java/org/apache/samza/sql/calcite/schema/TestAvroSchemaConverter.java
 PRE-CREATION 
  samza-sql-core/README.md PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Data.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/EntityName.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Relation.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Schema.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Stream.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Table.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Tuple.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/Operator.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorCallback.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorRouter.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorSpec.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SimpleOperator.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/IncomingMessageTuple.java
 PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/data/avro/AvroData.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/data/avro/AvroSchema.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlAvroSerde.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlAvroSerdeFactory.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlStringSerde.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlStringSerdeFactory.java
 PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/data/string/StringData.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/string/StringSchema.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/NoopOperatorCallback.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorFactoryImpl.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorImpl.java
 PRE-CREATION 
  
samza-sql

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-07-18 Thread Chris Pettitt

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/#review142421
---



The description says this is the initial draft implementation, but the title 
says initial draft for the APIs. I take it the latter is more accurate?

Some initial high-level thoughts:

1. We should use Java 8 constructs where possible (if we've moved to Java 8). 
For example, Function. If not, we should probably have some Function equivalent 
and various type specifications for map, flatMap, etc.
2. The stream operators don't seem to **do** anything - they primarily appear 
to hold metadata about something to be done. Are they intended only to be 
declarative? Are they also not intended to know directly how to build 
themselves, but rather you would run some graph processor (with knowledge about 
each operator type) to build out a real processor? This is pretty fundamental 
to the design. If there is a doc that covers this let me know; otherwise it 
would be super helpful to be on the same page about the end goal.
3. If bullet 2 is correct, how would you navigate through the graph? It would 
seem that you would need some way to navigate `(source stream, operator)` 
tuples?
4. Related to bullet 3, why do the operators know about their output streams? 
Abstractly aren't they totally independent in the sense that I could apply the 
same operator to multiple input streams to produce a corresponding number of 
output streams?
5. It looks like you can compose graphs in two ways: directly using the 
operators or using the interfaces on DataStream. I would choose one or the 
other and use appropriate hiding mechanisms to expose just the API the user 
should be concerned with. If you go with the latter I would extract interfaces 
for the operators (if that is even necessary) into a public package and hide 
everything else in a package private namespace.
6. I suspect you want to limit the ability to create custom operators (at least 
if the assumption about how graph walking would work in bullet 2 holds), so 
StreamOperator's constructor probably needs to be package private.

I suspect some over arching docs or a bluejeans session would be very helpful 
in allowing me to dig deeper into this.


samza-operator/src/main/java/org/apache/samza/operators/api/FlatMapper.java 
(line 34)


Have we moved to Java 8 yet? If so, it would be preferable to use the Java 
8 constructs if possible. If not, I think this can be generalized a little more 
nicely as a function from T to a Collection of R instead of having a separate 
flat map function type.

This comment can be applied generally wherever we're providing "functions"


- Chris Pettitt


On July 13, 2016, 8:54 a.m., Yi Pan (Data Infrastructure) wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/47835/
> ---
> 
> (Updated July 13, 2016, 8:54 a.m.)
> 
> 
> Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
> Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.
> 
> 
> Bugs: SAMZA-914
> https://issues.apache.org/jira/browse/SAMZA-914
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> SAMZA-914: initial draft implementation of operator programming API.
> 
> 
> Diffs
> -
> 
>   build.gradle 16facbbf4dff378c561461786ff186bd9eed 
>   gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
>   samza-operator/src/main/java/org/apache/samza/operators/api/FlatMapper.java 
> PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Mapper.java 
> PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Scanner.java 
> PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/api/Sink.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/data/DataStream.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/join/Joiner.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/window/SessionWindow.java
>  PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/api/window/Window.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/operators/impl/Pipeline.java 
> PRE-CREATION 
>   
> samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
>  PRE-CREATION 
>   samza-operator/src/main/java/org/apache/samza/task/DataStreamTask.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/TestScanner.java 
> PRE-CREATION 
>   
> samza-operator/src/test/java/org/apache/samza/operators/api/data/TestDataStream.java
>

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-07-13 Thread Yi Pan (Data Infrastructure)

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/
---

(Updated July 13, 2016, 8:54 a.m.)


Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.


Changes
---

Updated RB

- removed process() method from DataStream and Transformer class. Operator APIs 
are now mainly just mappers (map, flatMap, filter, scan, sink) and reducers 
(window aggregator, joiner).
- added more Javadoc for API methods
- moved IncomingMessageTuple to api package
- moved InputSystemStream as an embedded class SystemDataStream in DataStream
- include fix to AvroSchema and AvroData s.t. the source code is avro 1.4 
compatible


Bugs: SAMZA-914
https://issues.apache.org/jira/browse/SAMZA-914


Repository: samza


Description
---

SAMZA-914: initial draft implementation of operator programming API.


Diffs (updated)
-

  build.gradle 16facbbf4dff378c561461786ff186bd9eed 
  gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
  samza-operator/src/main/java/org/apache/samza/operators/api/FlatMapper.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/Mapper.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/Scanner.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/Sink.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/data/DataStream.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/join/Joiner.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/window/SessionWindow.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/window/Window.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/impl/Pipeline.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/task/DataStreamTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/operators/api/TestScanner.java 
PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/api/data/TestDataStream.java
 PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/DataStreamJoinTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/DataStreamSplitTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/DataStreamUserTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/TestDataStreamTask.java 
PRE-CREATION 
  
samza-sql-calcite/src/test/java/org/apache/samza/sql/calcite/schema/TestAvroSchemaConverter.java
 PRE-CREATION 
  samza-sql-core/README.md PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Data.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/EntityName.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Relation.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Schema.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Stream.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Table.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Tuple.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/Operator.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorCallback.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorRouter.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorSpec.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SimpleOperator.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/IncomingMessageTuple.java
 PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/data/avro/AvroData.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/data/avro/AvroSchema.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlAvroSerde.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlAvroSerdeFactory.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlStringSerde.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlStringSerdeFactory.java
 PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/data/string/StringData.java 
PRE-CREATION 
  
samza-s

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-06-07 Thread Yi Pan (Data Infrastructure)

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/
---

(Updated June 7, 2016, 7:42 p.m.)


Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.


Changes
---

Revert the change to Scanner API


Bugs: SAMZA-914
https://issues.apache.org/jira/browse/SAMZA-914


Repository: samza


Description
---

SAMZA-914: initial draft implementation of operator programming API.


Diffs (updated)
-

  build.gradle 16facbbf4dff378c561461786ff186bd9eed 
  gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
  samza-operator/src/main/java/org/apache/samza/operators/api/FlatMapper.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/Scanner.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/Transformer.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/data/DataStream.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/join/Joiner.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/window/Limit.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/window/SessionWindow.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/window/Window.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/impl/Pipeline.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/ProcessorContext.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/task/DataStreamTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/operators/api/TestScanner.java 
PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/api/data/TestDataStream.java
 PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/DataStreamJoinTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/DataStreamSplitTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/DataStreamUserTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/TestDataStreamTask.java 
PRE-CREATION 
  
samza-sql-calcite/src/test/java/org/apache/samza/sql/calcite/schema/TestAvroSchemaConverter.java
 PRE-CREATION 
  samza-sql-core/README.md PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Data.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/EntityName.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Relation.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Schema.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Stream.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Table.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Tuple.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/Operator.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorCallback.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorRouter.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorSpec.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SimpleOperator.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/IncomingMessageTuple.java
 PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/data/avro/AvroData.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/data/avro/AvroSchema.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlAvroSerde.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlAvroSerdeFactory.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlStringSerde.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlStringSerdeFactory.java
 PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/data/string/StringData.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/string/StringSchema.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/NoopOperatorCallback.java
 PRE-CREATION 
 

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-06-07 Thread Yi Pan (Data Infrastructure)

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/
---

(Updated June 7, 2016, 6:36 p.m.)


Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.


Changes
---

Fixing unit tests


Bugs: SAMZA-914
https://issues.apache.org/jira/browse/SAMZA-914


Repository: samza


Description
---

SAMZA-914: initial draft implementation of operator programming API.


Diffs (updated)
-

  build.gradle 16facbbf4dff378c561461786ff186bd9eed 
  gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
  samza-operator/src/main/java/org/apache/samza/operators/api/FlatMapper.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/Scanner.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/Transformer.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/data/DataStream.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/join/Joiner.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/window/Limit.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/window/SessionWindow.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/window/Window.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/impl/Pipeline.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/ProcessorContext.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/task/DataStreamTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/operators/api/TestScanner.java 
PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/api/data/TestDataStream.java
 PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/DataStreamJoinTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/DataStreamSplitTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/DataStreamUserTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/TestDataStreamTask.java 
PRE-CREATION 
  
samza-sql-calcite/src/test/java/org/apache/samza/sql/calcite/schema/TestAvroSchemaConverter.java
 PRE-CREATION 
  samza-sql-core/README.md PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Data.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/EntityName.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Relation.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Schema.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Stream.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Table.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Tuple.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/Operator.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorCallback.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorRouter.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorSpec.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SimpleOperator.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/IncomingMessageTuple.java
 PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/data/avro/AvroData.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/data/avro/AvroSchema.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlAvroSerde.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlAvroSerdeFactory.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlStringSerde.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlStringSerdeFactory.java
 PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/data/string/StringData.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/string/StringSchema.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/NoopOperatorCallback.java
 PRE-CREATION 
  
samza-sql-cor

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-06-07 Thread Yi Pan (Data Infrastructure)

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/
---

(Updated June 7, 2016, 5:24 p.m.)


Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.


Bugs: SAMZA-914
https://issues.apache.org/jira/browse/SAMZA-914


Repository: samza


Description
---

SAMZA-914: initial draft implementation of operator programming API.


Diffs (updated)
-

  build.gradle 16facbbf4dff378c561461786ff186bd9eed 
  gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
  samza-operator/src/main/java/org/apache/samza/operators/api/FlatMapper.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/Scanner.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/Transformer.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/data/DataStream.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/join/Joiner.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/window/SessionWindow.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/window/Timeout.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/window/Window.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/impl/Pipeline.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/ProcessorContext.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/task/DataStreamTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/operators/api/TestScanner.java 
PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/api/data/TestDataStream.java
 PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/DataStreamJoinTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/DataStreamSplitTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/DataStreamUserTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/TestDataStreamTask.java 
PRE-CREATION 
  
samza-sql-calcite/src/test/java/org/apache/samza/sql/calcite/schema/TestAvroSchemaConverter.java
 PRE-CREATION 
  samza-sql-core/README.md PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Data.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/EntityName.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Relation.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Schema.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Stream.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Table.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Tuple.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/Operator.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorCallback.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorRouter.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorSpec.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SimpleOperator.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/IncomingMessageTuple.java
 PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/data/avro/AvroData.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/data/avro/AvroSchema.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlAvroSerde.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlAvroSerdeFactory.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlStringSerde.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlStringSerdeFactory.java
 PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/data/string/StringData.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/string/StringSchema.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/NoopOperatorCallback.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/s

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-06-07 Thread Yi Pan (Data Infrastructure)

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/
---

(Updated June 7, 2016, 5:20 p.m.)


Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.


Bugs: SAMZA-914
https://issues.apache.org/jira/browse/SAMZA-914


Repository: samza


Description
---

SAMZA-914: initial draft implementation of operator programming API.


Diffs (updated)
-

  build.gradle 16facbbf4dff378c561461786ff186bd9eed 
  gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
  samza-operator/README.md PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/Filter.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/FlatMapper.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/Mapper.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/Scanner.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/Sink.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/StreamOperator.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/Transformer.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/data/Data.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/data/DataStream.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/data/InputSystemStream.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/data/Offset.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/data/Schema.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/data/Tuple.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/join/Joiner.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/window/Limit.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/window/SessionWindow.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/window/Timeout.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/window/TriggerBuilder.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/window/Window.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/impl/Pipeline.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/ProcessorContext.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/data/IncomingMessageTuple.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/data/LongOffset.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/data/avro/AvroData.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/data/avro/AvroSchema.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/data/serializers/SqlAvroSerde.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/data/serializers/SqlAvroSerdeFactory.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/data/serializers/SqlStringSerde.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/data/serializers/SqlStringSerdeFactory.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/data/string/StringData.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/data/string/StringSchema.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/task/DataStreamTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/operators/api/TestScanner.java 
PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/api/data/TestDataStream.java
 PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/api/window/TestSessionWindow.java
 PRE-CREATION 
  
samza-operator/src/test/java/org/apache/samza/operators/impl/data/serializers/SqlAvroSerdeTest.java
 PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/DataStreamJoinTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/DataStreamSplitTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/DataStreamUserTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/TestDataStreamTask.java 
PRE-CREATION 
  
samza-sql-calcite/src/main/java

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-05-27 Thread Yi Pan (Data Infrastructure)

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/
---

(Updated May 28, 2016, 12:29 a.m.)


Review request for samza, Boris Shkolnik, Chris Pettitt, Chinmay Soman, Jake 
Maes, Navina Ramesh, Jagadish Venkatraman, and Xinyu Liu.


Changes
---

Re-posting the diff based on the head of samza-sql branch


Bugs: SAMZA-914
https://issues.apache.org/jira/browse/SAMZA-914


Repository: samza


Description
---

SAMZA-914: initial draft implementation of operator programming API.


Diffs (updated)
-

  build.gradle 16facbbf4dff378c561461786ff186bd9eed 
  gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
  samza-operator/src/main/java/org/apache/samza/operators/api/FlatMapper.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/Scanner.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/Transformer.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/data/DataStream.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/join/Joiner.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/window/SessionWindow.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/window/Window.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/impl/Pipeline.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/ProcessorContext.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/task/DataStreamTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/DataStreamJoinTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/DataStreamSplitTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/DataStreamUserTask.java 
PRE-CREATION 
  
samza-sql-calcite/src/test/java/org/apache/samza/sql/calcite/schema/TestAvroSchemaConverter.java
 PRE-CREATION 
  samza-sql-core/README.md PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Data.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/EntityName.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Relation.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Schema.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Stream.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Table.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Tuple.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/Operator.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorCallback.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorRouter.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorSpec.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SimpleOperator.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/IncomingMessageTuple.java
 PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/data/avro/AvroData.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/data/avro/AvroSchema.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlAvroSerde.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlAvroSerdeFactory.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlStringSerde.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlStringSerdeFactory.java
 PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/data/string/StringData.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/string/StringSchema.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/NoopOperatorCallback.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorFactoryImpl.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorImpl.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorSpec.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/SimpleRouter.java
 PRE-CREATION 
  
samza-sql-co

Re: Review Request 47835: SAMZA-914: Initial draft for Java programming APIs on operators supporting DAGs

2016-05-27 Thread Yi Pan (Data Infrastructure)

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/47835/
---

(Updated May 27, 2016, 11:14 p.m.)


Review request for samza.


Bugs: SAMZA-914
https://issues.apache.org/jira/browse/SAMZA-914


Repository: samza


Description
---

SAMZA-914: initial draft implementation of operator programming API.


Diffs
-

  build.gradle 16facbbf4dff378c561461786ff186bd9eed 
  gradle/dependency-versions.gradle 52e25aa53a1edc85d478b48898621b26508ad4bb 
  samza-operator/src/main/java/org/apache/samza/operators/api/FlatMapper.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/Scanner.java 
PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/Transformer.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/data/DataStream.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/operators/api/join/Joiner.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/window/SessionWindow.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/api/window/Window.java 
PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/ProcessorContext.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
 PRE-CREATION 
  
samza-operator/src/main/java/org/apache/samza/operators/impl/StreamPipeline.java
 PRE-CREATION 
  samza-operator/src/main/java/org/apache/samza/task/DataStreamTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/DataStreamJoinTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/DataStreamSplitTask.java 
PRE-CREATION 
  samza-operator/src/test/java/org/apache/samza/task/DataStreamUserTask.java 
PRE-CREATION 
  
samza-sql-calcite/src/test/java/org/apache/samza/sql/calcite/schema/TestAvroSchemaConverter.java
 PRE-CREATION 
  samza-sql-core/README.md PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Data.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/EntityName.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Relation.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Schema.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Stream.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Table.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/data/Tuple.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/Operator.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorCallback.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorRouter.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/OperatorSpec.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SimpleOperator.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/IncomingMessageTuple.java
 PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/data/avro/AvroData.java 
PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/data/avro/AvroSchema.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlAvroSerde.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlAvroSerdeFactory.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlStringSerde.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/serializers/SqlStringSerdeFactory.java
 PRE-CREATION 
  samza-sql-core/src/main/java/org/apache/samza/sql/data/string/StringData.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/data/string/StringSchema.java 
PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/NoopOperatorCallback.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorFactoryImpl.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorImpl.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorSpec.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/operators/factory/SimpleRouter.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/operators/join/StreamStreamJoin.java
 PRE-CREATION 
  
samza-sql-core/src/main/java/org/apache/samza/sql/operators/join/StreamStreamJoinSpec.java
 PRE-C