You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@samza.apache.org by "Yi Pan (Data Infrastructure)" <yi...@linkedin.com> on 2015/01/10 00:07:13 UTC

Review Request 29754: StreamSQL operator API draft - This is the first draft of the StreamSQL operator APIs - org.apache.samza.sql.api.* contains definitions of all interface classes - org.apache.samza.sql.operators.

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

Review request for samza.


Repository: samza


Description
-------

2nd draft of SQL operator API
- Simplified data object interface
   * NOTE that the following two issues are still pending revision: i) nested data struct / schema; ii) multiple schema in a single stream?
- Simplified operator constructor
   * provided simple constructor of build-in operators w/o the need to use spec object and factory
- Added runtime context to allow operators to send out their output
   * this reduced the types of the operators to only two: RelationOperator and TupleOperator
- Added optional routing context to connect operators to each other
- Added two examples: 
   * i) a fully specified application that generates operators via factory and automatically executes the operators via routing context
   * ii) a "random" application that construct operators on-fly and use the runtime context to get the intermediate results back


Diffs
-----

  build.gradle 38383bd9e3f0847d6088a4ea4c1ee6f3dcd1e430 
  gradle/dependency-versions.gradle 44dd42603e93788562fd64c68312570cee71a2aa 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/IncomingMessageTuple.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/OutgoingMessageTuple.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/Relation.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/Tuple.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/Operator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/RelationOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/TupleOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/routing/OperatorRoutingContext.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/spec/OperatorSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/task/InitSystemContext.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/task/RuntimeSystemContext.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorFactoryImpl.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionOp.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/relation/Join.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/relation/JoinSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStream.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStreamSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/BoundedTimeWindow.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowState.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/data/SystemInputTuple.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/operators/routing/SimpleRoutingContext.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/store/SqlContextManager.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/task/RandomOperatorTask.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/task/RoutableRuntimeContext.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/task/StoredRuntimeContext.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/task/StreamSqlTask.java PRE-CREATION 
  settings.gradle 3a01fd66359b8c79954ae8f34eeaf4b2e3fdc0b4 

Diff: https://reviews.apache.org/r/29754/diff/


Testing
-------


Thanks,

Yi Pan (Data Infrastructure)


Re: Review Request 29754: StreamSQL operator API draft

Posted by "Yi Pan (Data Infrastructure)" <yi...@linkedin.com>.

> On Jan. 12, 2015, 8:44 p.m., Milinda Pathirage wrote:
> > samza-sql/src/main/java/org/apache/samza/sql/api/operators/routing/OperatorRoutingContext.java, line 47
> > <https://reviews.apache.org/r/29754/diff/1/?file=815332#file815332line47>
> >
> >     Are we planning to provide any ordering guarantees for the returned iterator?

@Milinda, at this moment, no. I couldn't think of the most popular patterns of the order required from this iterators yet. Hence, rather prefer to leave it open for now. Any perticular ordering semantics that you have in mind?


- Yi


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


On Jan. 10, 2015, 12:06 a.m., Yi Pan (Data Infrastructure) wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/29754/
> -----------------------------------------------------------
> 
> (Updated Jan. 10, 2015, 12:06 a.m.)
> 
> 
> Review request for samza, Chris Riccomini, Jay Kreps, Milinda Pathirage, Navina Ramesh, and Naveen Somasundaram.
> 
> 
> Bugs: SAMZA-482
>     https://issues.apache.org/jira/browse/SAMZA-482
> 
> 
> Repository: samza
> 
> 
> Description
> -------
> 
> 2nd draft of SQL operator API
> - Simplified data object interface
>    * NOTE that the following two issues are still pending revision: i) nested data struct / schema; ii) multiple schema in a single stream?
> - Simplified operator constructor
>    * provided simple constructor of build-in operators w/o the need to use spec object and factory
> - Added runtime context to allow operators to send out their output
>    * this reduced the types of the operators to only two: RelationOperator and TupleOperator
> - Added optional routing context to connect operators to each other
> - Added two examples: 
>    * i) a fully specified application that generates operators via factory and automatically executes the operators via routing context
>    * ii) a "random" application that construct operators on-fly and use the runtime context to get the intermediate results back
> 
> NOTE: there is still cleanups to be done to add java docs to the code and there will be more updates coming.
>   
> 
> 
> Diffs
> -----
> 
>   build.gradle 38383bd9e3f0847d6088a4ea4c1ee6f3dcd1e430 
>   gradle/dependency-versions.gradle 44dd42603e93788562fd64c68312570cee71a2aa 
>   samza-sql/src/main/java/org/apache/samza/sql/api/data/IncomingMessageTuple.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/data/OutgoingMessageTuple.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/data/Relation.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/data/Tuple.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/Operator.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/RelationOperator.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/TupleOperator.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/routing/OperatorRoutingContext.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/spec/OperatorSpec.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/task/InitSystemContext.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/task/RuntimeSystemContext.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperator.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorFactoryImpl.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionOp.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionSpec.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/relation/Join.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/relation/JoinSpec.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStream.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStreamSpec.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/window/BoundedTimeWindow.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowSpec.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowState.java PRE-CREATION 
>   samza-sql/src/test/java/org/apache/samza/sql/data/SystemInputTuple.java PRE-CREATION 
>   samza-sql/src/test/java/org/apache/samza/sql/operators/routing/SimpleRoutingContext.java PRE-CREATION 
>   samza-sql/src/test/java/org/apache/samza/sql/store/SqlContextManager.java PRE-CREATION 
>   samza-sql/src/test/java/org/apache/samza/sql/task/RandomOperatorTask.java PRE-CREATION 
>   samza-sql/src/test/java/org/apache/samza/sql/task/RoutableRuntimeContext.java PRE-CREATION 
>   samza-sql/src/test/java/org/apache/samza/sql/task/StoredRuntimeContext.java PRE-CREATION 
>   samza-sql/src/test/java/org/apache/samza/sql/task/StreamSqlTask.java PRE-CREATION 
>   settings.gradle 3a01fd66359b8c79954ae8f34eeaf4b2e3fdc0b4 
> 
> Diff: https://reviews.apache.org/r/29754/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Yi Pan (Data Infrastructure)
> 
>


Re: Review Request 29754: StreamSQL operator API draft

Posted by Milinda Pathirage <mi...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/29754/#review67697
-----------------------------------------------------------



samza-sql/src/main/java/org/apache/samza/sql/api/operators/routing/OperatorRoutingContext.java
<https://reviews.apache.org/r/29754/#comment111782>

    Are we planning to provide any ordering guarantees for the returned iterator?


- Milinda Pathirage


On Jan. 10, 2015, 12:06 a.m., Yi Pan (Data Infrastructure) wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/29754/
> -----------------------------------------------------------
> 
> (Updated Jan. 10, 2015, 12:06 a.m.)
> 
> 
> Review request for samza, Chris Riccomini, Jay Kreps, Milinda Pathirage, Navina Ramesh, and Naveen Somasundaram.
> 
> 
> Bugs: SAMZA-482
>     https://issues.apache.org/jira/browse/SAMZA-482
> 
> 
> Repository: samza
> 
> 
> Description
> -------
> 
> 2nd draft of SQL operator API
> - Simplified data object interface
>    * NOTE that the following two issues are still pending revision: i) nested data struct / schema; ii) multiple schema in a single stream?
> - Simplified operator constructor
>    * provided simple constructor of build-in operators w/o the need to use spec object and factory
> - Added runtime context to allow operators to send out their output
>    * this reduced the types of the operators to only two: RelationOperator and TupleOperator
> - Added optional routing context to connect operators to each other
> - Added two examples: 
>    * i) a fully specified application that generates operators via factory and automatically executes the operators via routing context
>    * ii) a "random" application that construct operators on-fly and use the runtime context to get the intermediate results back
> 
> NOTE: there is still cleanups to be done to add java docs to the code and there will be more updates coming.
>   
> 
> 
> Diffs
> -----
> 
>   build.gradle 38383bd9e3f0847d6088a4ea4c1ee6f3dcd1e430 
>   gradle/dependency-versions.gradle 44dd42603e93788562fd64c68312570cee71a2aa 
>   samza-sql/src/main/java/org/apache/samza/sql/api/data/IncomingMessageTuple.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/data/OutgoingMessageTuple.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/data/Relation.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/data/Tuple.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/Operator.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/RelationOperator.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/TupleOperator.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/routing/OperatorRoutingContext.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/spec/OperatorSpec.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/task/InitSystemContext.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/task/RuntimeSystemContext.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperator.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorFactoryImpl.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionOp.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionSpec.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/relation/Join.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/relation/JoinSpec.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStream.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStreamSpec.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/window/BoundedTimeWindow.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowSpec.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowState.java PRE-CREATION 
>   samza-sql/src/test/java/org/apache/samza/sql/data/SystemInputTuple.java PRE-CREATION 
>   samza-sql/src/test/java/org/apache/samza/sql/operators/routing/SimpleRoutingContext.java PRE-CREATION 
>   samza-sql/src/test/java/org/apache/samza/sql/store/SqlContextManager.java PRE-CREATION 
>   samza-sql/src/test/java/org/apache/samza/sql/task/RandomOperatorTask.java PRE-CREATION 
>   samza-sql/src/test/java/org/apache/samza/sql/task/RoutableRuntimeContext.java PRE-CREATION 
>   samza-sql/src/test/java/org/apache/samza/sql/task/StoredRuntimeContext.java PRE-CREATION 
>   samza-sql/src/test/java/org/apache/samza/sql/task/StreamSqlTask.java PRE-CREATION 
>   settings.gradle 3a01fd66359b8c79954ae8f34eeaf4b2e3fdc0b4 
> 
> Diff: https://reviews.apache.org/r/29754/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Yi Pan (Data Infrastructure)
> 
>


Re: Review Request 29754: StreamSQL operator API draft

Posted by "Yi Pan (Data Infrastructure)" <yi...@linkedin.com>.

> On Jan. 17, 2015, 2:05 a.m., Chris Riccomini wrote:
> > Some of these interfaces (RuntimeSystemContext, InitSystemContext, Relation, Operator) seem very similar to existing interfaces (MessageCollector, TaskContext, KeyValueStorageEngine, StreamTask). Have you tried using existing interfaces, and just extending them where needed?

@Chris, good point. I have not tried that yet. I actually started to integrate the Relation class w/ the KeyValueStore already when I am writing some test code to implement the InitSystemContext. I would follow that suggestion.


> On Jan. 17, 2015, 2:05 a.m., Chris Riccomini wrote:
> > samza-sql/src/main/java/org/apache/samza/sql/api/data/IncomingMessageTuple.java, line 52
> > <https://reviews.apache.org/r/29754/diff/1-2/?file=815324#file815324line52>
> >
> >     This method seems redundant, since IncomingMessageEnvelope.getSystemStreamPartition() already exists.

Sure. I added that simply for convenience. Let me remove it.


> On Jan. 17, 2015, 2:05 a.m., Chris Riccomini wrote:
> > samza-sql/src/main/java/org/apache/samza/sql/api/data/OutgoingMessageTuple.java, line 52
> > <https://reviews.apache.org/r/29754/diff/1-2/?file=815325#file815325line52>
> >
> >     Seems like a dupe, since OutgoingMessageEnvelope can provide this information.

Yes. Same here. Will remove.


> On Jan. 17, 2015, 2:05 a.m., Chris Riccomini wrote:
> > samza-sql/src/main/java/org/apache/samza/sql/api/data/OutgoingMessageTuple.java, line 60
> > <https://reviews.apache.org/r/29754/diff/1-2/?file=815325#file815325line60>
> >
> >     Seems like a dupe, since OutgoingMessageEnvelope can provide this information.

Sure.


> On Jan. 17, 2015, 2:05 a.m., Chris Riccomini wrote:
> > samza-sql/src/main/java/org/apache/samza/sql/api/data/Tuple.java, line 35
> > <https://reviews.apache.org/r/29754/diff/1-2/?file=815327#file815327line35>
> >
> >     How is this related to IncomingMessageTuple and OutgoingMessageTuple?

Tuple.getMessage() should return the actual message body (e.g. IncomingMessageEnvelope.getMessage()). Actually, with the exercise I was doing with the schema, I would think that Tuple.getMessage() should return the generic data object Data in review #29994.


> On Jan. 17, 2015, 2:05 a.m., Chris Riccomini wrote:
> > samza-sql/src/main/java/org/apache/samza/sql/api/operators/Operator.java, line 68
> > <https://reviews.apache.org/r/29754/diff/1-2/?file=815328#file815328line68>
> >
> >     Should this return "SELECT" for all SelectOperators? Or "SELECT-{guid}", where each ID is unique even amongst operators of the same type?

It should be the second case. The ID of the operator is the unique identifier of the instance of the operator in the task.


> On Jan. 17, 2015, 2:05 a.m., Chris Riccomini wrote:
> > samza-sql/src/main/java/org/apache/samza/sql/api/operators/spec/OperatorSpec.java, line 42
> > <https://reviews.apache.org/r/29754/diff/1-2/?file=815333#file815333line42>
> >
> >     Do we need Operator.getId if we have this?

Functionally, it is redundant. I added it for convenience. Let me remove it.


> On Jan. 17, 2015, 2:05 a.m., Chris Riccomini wrote:
> > samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorSpec.java, line 35
> > <https://reviews.apache.org/r/29754/diff/2/?file=818474#file818474line35>
> >
> >     Is it possible to have multiple outputs?

No. If we are thinking of supporting multiple outputs, the routing context API and the runtime context API would be more complicated. With SQL operators in mind, we only need to support a tree topology for execution. Let me think it over to see whether we can add support for arbitrary topology w/o adding too much complexity.


- Yi


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


On Jan. 13, 2015, 12:56 a.m., Yi Pan (Data Infrastructure) wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/29754/
> -----------------------------------------------------------
> 
> (Updated Jan. 13, 2015, 12:56 a.m.)
> 
> 
> Review request for samza, Chris Riccomini, Jay Kreps, Milinda Pathirage, Navina Ramesh, and Naveen Somasundaram.
> 
> 
> Bugs: SAMZA-482
>     https://issues.apache.org/jira/browse/SAMZA-482
> 
> 
> Repository: samza
> 
> 
> Description
> -------
> 
> Overview of the proposal:
> - 4 different types of operators implemented via 2 interface classes:
>    * stream-to-relation operators implementing TupleOperator interface and generate Relation output. E.g. Window operators
>    * stream-to-stream operators implementing TupleOperator interface and generate Tuple output. E.g. Partition operators
>    * relation-to-relation operators implementing RelationOperator interface and generate Relation output. E.g. All relational algebra operators, s.t. join, where, group-by, select, etc.
>    * relation-to-stream operators implementing RelationOperator interface and generate Tuple output. E.g. Istream or Dstream operators
> 
> Those operators are connected via the following two context interface classes:
> - RuntimeSystemContext which provides a context interface for the operators to send their output to
> - OperatorRoutingContext which provides the connection interface between the operators
> 
> In the example, we have enabled two execution models via the above two context classes:
> - RoutableRuntimeContext that uses the routing information from a OperatorRoutingContext and directly invoking the next operator when the current operator send its output via the RoutableRuntimeContext
> - StoredRuntimeContext that provides a storage for each operator's outputs that stored the output when the current operator send its output via StoredRuntimeContext. 
> 
> Then, it is up to the programmer to query the StoredRuntimeContext to get the operator's output and proceed w/ the next steps
> The first execution model allows the integration w/ future SQL parser and planner to automatically run a task, while the second model allows a random programmer to use the operators from the library in a random context.
>   
>   
> 2nd draft of SQL operator API
> - Simplified data object interface
>    * NOTE that the following two issues are still pending revision: i) nested data struct / schema; ii) multiple schema in a single stream?
> - Simplified operator constructor
>    * provided simple constructor of build-in operators w/o the need to use spec object and factory
> - Added runtime context to allow operators to send out their output
>    * this reduced the types of the operators to only two: RelationOperator and TupleOperator
> - Added optional routing context to connect operators to each other
> - Added two examples: 
>    * i) a fully specified application that generates operators via factory and automatically executes the operators via routing context
>    * ii) a "random" application that construct operators on-fly and use the runtime context to get the intermediate results back
> 
> 
> Diffs
> -----
> 
>   build.gradle 38383bd9e3f0847d6088a4ea4c1ee6f3dcd1e430 
>   gradle/dependency-versions.gradle 44dd42603e93788562fd64c68312570cee71a2aa 
>   samza-sql/src/main/java/org/apache/samza/sql/api/data/IncomingMessageTuple.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/data/OutgoingMessageTuple.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/data/Relation.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/data/Tuple.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/Operator.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/RelationOperator.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/TupleOperator.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/routing/OperatorRoutingContext.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/spec/OperatorSpec.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/task/InitSystemContext.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/task/RuntimeSystemContext.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperator.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorFactoryImpl.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorSpec.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionOp.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionSpec.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/relation/Join.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/relation/JoinSpec.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStream.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStreamSpec.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/window/BoundedTimeWindow.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowSpec.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowState.java PRE-CREATION 
>   samza-sql/src/test/java/org/apache/samza/sql/data/SystemInputTuple.java PRE-CREATION 
>   samza-sql/src/test/java/org/apache/samza/sql/operators/routing/SimpleRoutingContext.java PRE-CREATION 
>   samza-sql/src/test/java/org/apache/samza/sql/store/SqlContextManager.java PRE-CREATION 
>   samza-sql/src/test/java/org/apache/samza/sql/task/RandomOperatorTask.java PRE-CREATION 
>   samza-sql/src/test/java/org/apache/samza/sql/task/RoutableRuntimeContext.java PRE-CREATION 
>   samza-sql/src/test/java/org/apache/samza/sql/task/StoredRuntimeContext.java PRE-CREATION 
>   samza-sql/src/test/java/org/apache/samza/sql/task/StreamSqlTask.java PRE-CREATION 
>   settings.gradle 3a01fd66359b8c79954ae8f34eeaf4b2e3fdc0b4 
> 
> Diff: https://reviews.apache.org/r/29754/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Yi Pan (Data Infrastructure)
> 
>


Re: Review Request 29754: StreamSQL operator API draft

Posted by Chris Riccomini <cr...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/29754/#review68476
-----------------------------------------------------------


Some of these interfaces (RuntimeSystemContext, InitSystemContext, Relation, Operator) seem very similar to existing interfaces (MessageCollector, TaskContext, KeyValueStorageEngine, StreamTask). Have you tried using existing interfaces, and just extending them where needed?


samza-sql/src/main/java/org/apache/samza/sql/api/data/IncomingMessageTuple.java
<https://reviews.apache.org/r/29754/#comment112715>

    This method seems redundant, since IncomingMessageEnvelope.getSystemStreamPartition() already exists.



samza-sql/src/main/java/org/apache/samza/sql/api/data/OutgoingMessageTuple.java
<https://reviews.apache.org/r/29754/#comment112716>

    Seems like a dupe, since OutgoingMessageEnvelope can provide this information.



samza-sql/src/main/java/org/apache/samza/sql/api/data/OutgoingMessageTuple.java
<https://reviews.apache.org/r/29754/#comment112717>

    Seems like a dupe, since OutgoingMessageEnvelope can provide this information.



samza-sql/src/main/java/org/apache/samza/sql/api/data/Tuple.java
<https://reviews.apache.org/r/29754/#comment112728>

    How is this related to IncomingMessageTuple and OutgoingMessageTuple?



samza-sql/src/main/java/org/apache/samza/sql/api/operators/Operator.java
<https://reviews.apache.org/r/29754/#comment112735>

    Should this return "SELECT" for all SelectOperators? Or "SELECT-{guid}", where each ID is unique even amongst operators of the same type?



samza-sql/src/main/java/org/apache/samza/sql/api/operators/spec/OperatorSpec.java
<https://reviews.apache.org/r/29754/#comment112740>

    Do we need Operator.getId if we have this?



samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorSpec.java
<https://reviews.apache.org/r/29754/#comment112746>

    Is it possible to have multiple outputs?


- Chris Riccomini


On Jan. 13, 2015, 12:56 a.m., Yi Pan (Data Infrastructure) wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/29754/
> -----------------------------------------------------------
> 
> (Updated Jan. 13, 2015, 12:56 a.m.)
> 
> 
> Review request for samza, Chris Riccomini, Jay Kreps, Milinda Pathirage, Navina Ramesh, and Naveen Somasundaram.
> 
> 
> Bugs: SAMZA-482
>     https://issues.apache.org/jira/browse/SAMZA-482
> 
> 
> Repository: samza
> 
> 
> Description
> -------
> 
> Overview of the proposal:
> - 4 different types of operators implemented via 2 interface classes:
>    * stream-to-relation operators implementing TupleOperator interface and generate Relation output. E.g. Window operators
>    * stream-to-stream operators implementing TupleOperator interface and generate Tuple output. E.g. Partition operators
>    * relation-to-relation operators implementing RelationOperator interface and generate Relation output. E.g. All relational algebra operators, s.t. join, where, group-by, select, etc.
>    * relation-to-stream operators implementing RelationOperator interface and generate Tuple output. E.g. Istream or Dstream operators
> 
> Those operators are connected via the following two context interface classes:
> - RuntimeSystemContext which provides a context interface for the operators to send their output to
> - OperatorRoutingContext which provides the connection interface between the operators
> 
> In the example, we have enabled two execution models via the above two context classes:
> - RoutableRuntimeContext that uses the routing information from a OperatorRoutingContext and directly invoking the next operator when the current operator send its output via the RoutableRuntimeContext
> - StoredRuntimeContext that provides a storage for each operator's outputs that stored the output when the current operator send its output via StoredRuntimeContext. 
> 
> Then, it is up to the programmer to query the StoredRuntimeContext to get the operator's output and proceed w/ the next steps
> The first execution model allows the integration w/ future SQL parser and planner to automatically run a task, while the second model allows a random programmer to use the operators from the library in a random context.
>   
>   
> 2nd draft of SQL operator API
> - Simplified data object interface
>    * NOTE that the following two issues are still pending revision: i) nested data struct / schema; ii) multiple schema in a single stream?
> - Simplified operator constructor
>    * provided simple constructor of build-in operators w/o the need to use spec object and factory
> - Added runtime context to allow operators to send out their output
>    * this reduced the types of the operators to only two: RelationOperator and TupleOperator
> - Added optional routing context to connect operators to each other
> - Added two examples: 
>    * i) a fully specified application that generates operators via factory and automatically executes the operators via routing context
>    * ii) a "random" application that construct operators on-fly and use the runtime context to get the intermediate results back
> 
> 
> Diffs
> -----
> 
>   build.gradle 38383bd9e3f0847d6088a4ea4c1ee6f3dcd1e430 
>   gradle/dependency-versions.gradle 44dd42603e93788562fd64c68312570cee71a2aa 
>   samza-sql/src/main/java/org/apache/samza/sql/api/data/IncomingMessageTuple.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/data/OutgoingMessageTuple.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/data/Relation.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/data/Tuple.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/Operator.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/RelationOperator.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/TupleOperator.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/routing/OperatorRoutingContext.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/spec/OperatorSpec.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/task/InitSystemContext.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/task/RuntimeSystemContext.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperator.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorFactoryImpl.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorSpec.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionOp.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionSpec.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/relation/Join.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/relation/JoinSpec.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStream.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStreamSpec.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/window/BoundedTimeWindow.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowSpec.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowState.java PRE-CREATION 
>   samza-sql/src/test/java/org/apache/samza/sql/data/SystemInputTuple.java PRE-CREATION 
>   samza-sql/src/test/java/org/apache/samza/sql/operators/routing/SimpleRoutingContext.java PRE-CREATION 
>   samza-sql/src/test/java/org/apache/samza/sql/store/SqlContextManager.java PRE-CREATION 
>   samza-sql/src/test/java/org/apache/samza/sql/task/RandomOperatorTask.java PRE-CREATION 
>   samza-sql/src/test/java/org/apache/samza/sql/task/RoutableRuntimeContext.java PRE-CREATION 
>   samza-sql/src/test/java/org/apache/samza/sql/task/StoredRuntimeContext.java PRE-CREATION 
>   samza-sql/src/test/java/org/apache/samza/sql/task/StreamSqlTask.java PRE-CREATION 
>   settings.gradle 3a01fd66359b8c79954ae8f34eeaf4b2e3fdc0b4 
> 
> Diff: https://reviews.apache.org/r/29754/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Yi Pan (Data Infrastructure)
> 
>


Re: Review Request 29754: StreamSQL operator API draft

Posted by "Yi Pan (Data Infrastructure)" <yi...@linkedin.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/29754/
-----------------------------------------------------------

(Updated Jan. 31, 2015, 7:49 a.m.)


Review request for samza, Chris Riccomini, Jay Kreps, Milinda Pathirage, Navina Ramesh, and Naveen Somasundaram.


Changes
-------

Updated RB description


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


Repository: samza


Description (updated)
-------

Overview of the proposal:
- 4 different types of operators implemented via 2 interface classes:
   * stream-to-relation operators implementing TupleOperator interface and generate Relation output. E.g. Window operators
   * stream-to-stream operators implementing TupleOperator interface and generate Tuple output. E.g. Partition operators
   * relation-to-relation operators implementing RelationOperator interface and generate Relation output. E.g. All relational algebra operators, s.t. join, where, group-by, select, etc.
   * relation-to-stream operators implementing RelationOperator interface and generate Tuple output. E.g. Istream or Dstream operators

Those operators are connected via the following two context interface classes:
- SqlMessageCollector which provides a MessageCollector interface for the operators to send their output to
- OperatorRouter which provides the connection interface between the operators

In the example, we have enabled two execution models via the above two context classes:
- OperatorMessageCollector that uses the routing information from a OperatorRouter and directly invoking the next operator when the current operator send its output via the OperatorMessageCollector
- StoreMessageCollector that provides a storage for each operator's outputs that stored the output when the current operator send its output via StoreMessageCollector. 

Then, it is up to the programmer to query the StoreMessageCollector to get the operator's output and proceed w/ the next steps
The first execution model allows the integration w/ future SQL parser and planner to automatically run a task, while the second model allows a random programmer to use the operators from the library in a random context.


Diffs
-----

  build.gradle 7a40ad4ae916610186848c06c4577e7067de98ee 
  gradle/dependency-versions.gradle 44dd42603e93788562fd64c68312570cee71a2aa 
  samza-sql/README PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/EntityName.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/Relation.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/Tuple.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/Operator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/RelationOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/TupleOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/spec/OperatorSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/router/OperatorRouter.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/data/IncomingMessageTuple.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorFactoryImpl.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionOp.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/relation/Join.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/relation/JoinSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStream.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStreamSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/BoundedTimeWindow.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowState.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/router/SimpleRouter.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/task/sql/OperatorMessageCollector.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/task/sql/SqlMessageCollector.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/task/sql/StoreMessageCollector.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/task/sql/RandomOperatorTask.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/task/sql/StreamSqlTask.java PRE-CREATION 
  settings.gradle 3a01fd66359b8c79954ae8f34eeaf4b2e3fdc0b4 

Diff: https://reviews.apache.org/r/29754/diff/


Testing
-------

run ./bin/check-all.sh passed


Thanks,

Yi Pan (Data Infrastructure)


Re: Review Request 29754: StreamSQL operator API draft

Posted by "Yi Pan (Data Infrastructure)" <yi...@linkedin.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/29754/
-----------------------------------------------------------

(Updated Jan. 31, 2015, 7:47 a.m.)


Review request for samza, Chris Riccomini, Jay Kreps, Milinda Pathirage, Navina Ramesh, and Naveen Somasundaram.


Changes
-------

Fixed more comments and variables. Removed -d64 in test


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


Repository: samza


Description
-------

Overview of the proposal:
- 4 different types of operators implemented via 2 interface classes:
   * stream-to-relation operators implementing TupleOperator interface and generate Relation output. E.g. Window operators
   * stream-to-stream operators implementing TupleOperator interface and generate Tuple output. E.g. Partition operators
   * relation-to-relation operators implementing RelationOperator interface and generate Relation output. E.g. All relational algebra operators, s.t. join, where, group-by, select, etc.
   * relation-to-stream operators implementing RelationOperator interface and generate Tuple output. E.g. Istream or Dstream operators

Those operators are connected via the following two context interface classes:
- RuntimeSystemContext which provides a context interface for the operators to send their output to
- OperatorRoutingContext which provides the connection interface between the operators

In the example, we have enabled two execution models via the above two context classes:
- RoutableRuntimeContext that uses the routing information from a OperatorRoutingContext and directly invoking the next operator when the current operator send its output via the RoutableRuntimeContext
- StoredRuntimeContext that provides a storage for each operator's outputs that stored the output when the current operator send its output via StoredRuntimeContext. 

Then, it is up to the programmer to query the StoredRuntimeContext to get the operator's output and proceed w/ the next steps
The first execution model allows the integration w/ future SQL parser and planner to automatically run a task, while the second model allows a random programmer to use the operators from the library in a random context.
  
  
2nd draft of SQL operator API
- Simplified data object interface
   * NOTE that the following two issues are still pending revision: i) nested data struct / schema; ii) multiple schema in a single stream?
- Simplified operator constructor
   * provided simple constructor of build-in operators w/o the need to use spec object and factory
- Added runtime context to allow operators to send out their output
   * this reduced the types of the operators to only two: RelationOperator and TupleOperator
- Added optional routing context to connect operators to each other
- Added two examples: 
   * i) a fully specified application that generates operators via factory and automatically executes the operators via routing context
   * ii) a "random" application that construct operators on-fly and use the runtime context to get the intermediate results back


Diffs (updated)
-----

  build.gradle 7a40ad4ae916610186848c06c4577e7067de98ee 
  gradle/dependency-versions.gradle 44dd42603e93788562fd64c68312570cee71a2aa 
  samza-sql/README PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/EntityName.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/Relation.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/Tuple.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/Operator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/RelationOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/TupleOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/spec/OperatorSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/router/OperatorRouter.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/data/IncomingMessageTuple.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorFactoryImpl.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionOp.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/relation/Join.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/relation/JoinSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStream.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStreamSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/BoundedTimeWindow.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowState.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/router/SimpleRouter.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/task/sql/OperatorMessageCollector.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/task/sql/SqlMessageCollector.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/task/sql/StoreMessageCollector.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/task/sql/RandomOperatorTask.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/task/sql/StreamSqlTask.java PRE-CREATION 
  settings.gradle 3a01fd66359b8c79954ae8f34eeaf4b2e3fdc0b4 

Diff: https://reviews.apache.org/r/29754/diff/


Testing
-------

run ./bin/check-all.sh passed


Thanks,

Yi Pan (Data Infrastructure)


Re: Review Request 29754: StreamSQL operator API draft

Posted by "Yi Pan (Data Infrastructure)" <yi...@linkedin.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/29754/
-----------------------------------------------------------

(Updated Jan. 31, 2015, 2:02 a.m.)


Review request for samza, Chris Riccomini, Jay Kreps, Milinda Pathirage, Navina Ramesh, and Naveen Somasundaram.


Changes
-------

Missed last bit of comment: removing -d 64 in build.gradle


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


Repository: samza


Description
-------

Overview of the proposal:
- 4 different types of operators implemented via 2 interface classes:
   * stream-to-relation operators implementing TupleOperator interface and generate Relation output. E.g. Window operators
   * stream-to-stream operators implementing TupleOperator interface and generate Tuple output. E.g. Partition operators
   * relation-to-relation operators implementing RelationOperator interface and generate Relation output. E.g. All relational algebra operators, s.t. join, where, group-by, select, etc.
   * relation-to-stream operators implementing RelationOperator interface and generate Tuple output. E.g. Istream or Dstream operators

Those operators are connected via the following two context interface classes:
- RuntimeSystemContext which provides a context interface for the operators to send their output to
- OperatorRoutingContext which provides the connection interface between the operators

In the example, we have enabled two execution models via the above two context classes:
- RoutableRuntimeContext that uses the routing information from a OperatorRoutingContext and directly invoking the next operator when the current operator send its output via the RoutableRuntimeContext
- StoredRuntimeContext that provides a storage for each operator's outputs that stored the output when the current operator send its output via StoredRuntimeContext. 

Then, it is up to the programmer to query the StoredRuntimeContext to get the operator's output and proceed w/ the next steps
The first execution model allows the integration w/ future SQL parser and planner to automatically run a task, while the second model allows a random programmer to use the operators from the library in a random context.
  
  
2nd draft of SQL operator API
- Simplified data object interface
   * NOTE that the following two issues are still pending revision: i) nested data struct / schema; ii) multiple schema in a single stream?
- Simplified operator constructor
   * provided simple constructor of build-in operators w/o the need to use spec object and factory
- Added runtime context to allow operators to send out their output
   * this reduced the types of the operators to only two: RelationOperator and TupleOperator
- Added optional routing context to connect operators to each other
- Added two examples: 
   * i) a fully specified application that generates operators via factory and automatically executes the operators via routing context
   * ii) a "random" application that construct operators on-fly and use the runtime context to get the intermediate results back


Diffs (updated)
-----

  build.gradle 7a40ad4ae916610186848c06c4577e7067de98ee 
  gradle/dependency-versions.gradle 44dd42603e93788562fd64c68312570cee71a2aa 
  samza-sql/README PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/EntityName.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/Relation.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/Tuple.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/Operator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/RelationOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/TupleOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/spec/OperatorSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/router/OperatorRouter.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/data/IncomingMessageTuple.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorFactoryImpl.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionOp.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/relation/Join.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/relation/JoinSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStream.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStreamSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/BoundedTimeWindow.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowState.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/router/SimpleRouter.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/task/sql/OperatorMessageCollector.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/task/sql/SqlMessageCollector.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/task/sql/StoreMessageCollector.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/task/sql/RandomOperatorTask.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/task/sql/StreamSqlTask.java PRE-CREATION 
  settings.gradle 3a01fd66359b8c79954ae8f34eeaf4b2e3fdc0b4 

Diff: https://reviews.apache.org/r/29754/diff/


Testing
-------

run ./bin/check-all.sh passed


Thanks,

Yi Pan (Data Infrastructure)


Re: Review Request 29754: StreamSQL operator API draft

Posted by "Yi Pan (Data Infrastructure)" <yi...@linkedin.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/29754/
-----------------------------------------------------------

(Updated Jan. 30, 2015, 8:34 p.m.)


Review request for samza, Chris Riccomini, Jay Kreps, Milinda Pathirage, Navina Ramesh, and Naveen Somasundaram.


Changes
-------

Revised based on the lastest comment.


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


Repository: samza


Description
-------

Overview of the proposal:
- 4 different types of operators implemented via 2 interface classes:
   * stream-to-relation operators implementing TupleOperator interface and generate Relation output. E.g. Window operators
   * stream-to-stream operators implementing TupleOperator interface and generate Tuple output. E.g. Partition operators
   * relation-to-relation operators implementing RelationOperator interface and generate Relation output. E.g. All relational algebra operators, s.t. join, where, group-by, select, etc.
   * relation-to-stream operators implementing RelationOperator interface and generate Tuple output. E.g. Istream or Dstream operators

Those operators are connected via the following two context interface classes:
- RuntimeSystemContext which provides a context interface for the operators to send their output to
- OperatorRoutingContext which provides the connection interface between the operators

In the example, we have enabled two execution models via the above two context classes:
- RoutableRuntimeContext that uses the routing information from a OperatorRoutingContext and directly invoking the next operator when the current operator send its output via the RoutableRuntimeContext
- StoredRuntimeContext that provides a storage for each operator's outputs that stored the output when the current operator send its output via StoredRuntimeContext. 

Then, it is up to the programmer to query the StoredRuntimeContext to get the operator's output and proceed w/ the next steps
The first execution model allows the integration w/ future SQL parser and planner to automatically run a task, while the second model allows a random programmer to use the operators from the library in a random context.
  
  
2nd draft of SQL operator API
- Simplified data object interface
   * NOTE that the following two issues are still pending revision: i) nested data struct / schema; ii) multiple schema in a single stream?
- Simplified operator constructor
   * provided simple constructor of build-in operators w/o the need to use spec object and factory
- Added runtime context to allow operators to send out their output
   * this reduced the types of the operators to only two: RelationOperator and TupleOperator
- Added optional routing context to connect operators to each other
- Added two examples: 
   * i) a fully specified application that generates operators via factory and automatically executes the operators via routing context
   * ii) a "random" application that construct operators on-fly and use the runtime context to get the intermediate results back


Diffs (updated)
-----

  build.gradle 7a40ad4ae916610186848c06c4577e7067de98ee 
  gradle/dependency-versions.gradle 44dd42603e93788562fd64c68312570cee71a2aa 
  samza-sql/README PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/EntityName.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/Relation.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/Tuple.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/Operator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/RelationOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/TupleOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/spec/OperatorSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/router/OperatorRouter.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/data/IncomingMessageTuple.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorFactoryImpl.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionOp.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/relation/Join.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/relation/JoinSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStream.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStreamSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/BoundedTimeWindow.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowState.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/router/SimpleRouter.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/task/sql/OperatorMessageCollector.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/task/sql/SqlMessageCollector.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/task/sql/StoreMessageCollector.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/task/sql/RandomOperatorTask.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/task/sql/StreamSqlTask.java PRE-CREATION 
  settings.gradle 3a01fd66359b8c79954ae8f34eeaf4b2e3fdc0b4 

Diff: https://reviews.apache.org/r/29754/diff/


Testing
-------

run ./bin/check-all.sh passed


Thanks,

Yi Pan (Data Infrastructure)


Re: Review Request 29754: StreamSQL operator API draft

Posted by "Yi Pan (Data Infrastructure)" <yi...@linkedin.com>.

> On Jan. 30, 2015, 6:04 p.m., Chris Riccomini wrote:
> > build.gradle, line 265
> > <https://reviews.apache.org/r/29754/diff/4/?file=840301#file840301line265>
> >
> >     Remove -d64 here. Tests should work on 32 bit machines.

Sure. I will update this.


> On Jan. 30, 2015, 6:04 p.m., Chris Riccomini wrote:
> > samza-sql/src/main/java/org/apache/samza/sql/api/data/Tuple.java, line 50
> > <https://reviews.apache.org/r/29754/diff/4/?file=840305#file840305line50>
> >
> >     This API is a little strange to me. I'm assuming that this is getting a field from the message, right?

Yes. Now with the Data API in schema, I think that we can remove this since getMessage() will return a Data object and user can get the field value from the Data object.


> On Jan. 30, 2015, 6:04 p.m., Chris Riccomini wrote:
> > samza-sql/src/main/java/org/apache/samza/sql/api/operators/routing/OperatorRoutingContext.java, line 53
> > <https://reviews.apache.org/r/29754/diff/4/?file=840310#file840310line53>
> >
> >     OperatorRouter?

Sounds good to me.


> On Jan. 30, 2015, 6:04 p.m., Chris Riccomini wrote:
> > samza-sql/src/main/java/org/apache/samza/sql/api/task/RuntimeSystemContext.java, line 35
> > <https://reviews.apache.org/r/29754/diff/4/?file=840312#file840312line35>
> >
> >     SqlMessageCollector? RoutingMessageCollector?

I would prefer SqlMessageCollector.


> On Jan. 30, 2015, 6:04 p.m., Chris Riccomini wrote:
> > samza-sql/src/main/java/org/apache/samza/sql/task/RoutableRuntimeContext.java, line 41
> > <https://reviews.apache.org/r/29754/diff/4/?file=840327#file840327line41>
> >
> >     OperatorMessageCollector?

Sounds good. Will change.


> On Jan. 30, 2015, 6:04 p.m., Chris Riccomini wrote:
> > samza-sql/src/main/java/org/apache/samza/sql/task/StoredRuntimeContext.java, line 37
> > <https://reviews.apache.org/r/29754/diff/4/?file=840328#file840328line37>
> >
> >     StoreMessageCollector?

Sounds good to me.


- Yi


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


On Jan. 29, 2015, 7:32 p.m., Yi Pan (Data Infrastructure) wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/29754/
> -----------------------------------------------------------
> 
> (Updated Jan. 29, 2015, 7:32 p.m.)
> 
> 
> Review request for samza, Chris Riccomini, Jay Kreps, Milinda Pathirage, Navina Ramesh, and Naveen Somasundaram.
> 
> 
> Bugs: SAMZA-482
>     https://issues.apache.org/jira/browse/SAMZA-482
> 
> 
> Repository: samza
> 
> 
> Description
> -------
> 
> Overview of the proposal:
> - 4 different types of operators implemented via 2 interface classes:
>    * stream-to-relation operators implementing TupleOperator interface and generate Relation output. E.g. Window operators
>    * stream-to-stream operators implementing TupleOperator interface and generate Tuple output. E.g. Partition operators
>    * relation-to-relation operators implementing RelationOperator interface and generate Relation output. E.g. All relational algebra operators, s.t. join, where, group-by, select, etc.
>    * relation-to-stream operators implementing RelationOperator interface and generate Tuple output. E.g. Istream or Dstream operators
> 
> Those operators are connected via the following two context interface classes:
> - RuntimeSystemContext which provides a context interface for the operators to send their output to
> - OperatorRoutingContext which provides the connection interface between the operators
> 
> In the example, we have enabled two execution models via the above two context classes:
> - RoutableRuntimeContext that uses the routing information from a OperatorRoutingContext and directly invoking the next operator when the current operator send its output via the RoutableRuntimeContext
> - StoredRuntimeContext that provides a storage for each operator's outputs that stored the output when the current operator send its output via StoredRuntimeContext. 
> 
> Then, it is up to the programmer to query the StoredRuntimeContext to get the operator's output and proceed w/ the next steps
> The first execution model allows the integration w/ future SQL parser and planner to automatically run a task, while the second model allows a random programmer to use the operators from the library in a random context.
>   
>   
> 2nd draft of SQL operator API
> - Simplified data object interface
>    * NOTE that the following two issues are still pending revision: i) nested data struct / schema; ii) multiple schema in a single stream?
> - Simplified operator constructor
>    * provided simple constructor of build-in operators w/o the need to use spec object and factory
> - Added runtime context to allow operators to send out their output
>    * this reduced the types of the operators to only two: RelationOperator and TupleOperator
> - Added optional routing context to connect operators to each other
> - Added two examples: 
>    * i) a fully specified application that generates operators via factory and automatically executes the operators via routing context
>    * ii) a "random" application that construct operators on-fly and use the runtime context to get the intermediate results back
> 
> 
> Diffs
> -----
> 
>   build.gradle 7a40ad4ae916610186848c06c4577e7067de98ee 
>   gradle/dependency-versions.gradle 44dd42603e93788562fd64c68312570cee71a2aa 
>   samza-sql/src/main/java/org/apache/samza/sql/api/data/EntityName.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/data/Relation.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/data/Tuple.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/Operator.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/RelationOperator.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/TupleOperator.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/routing/OperatorRoutingContext.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/spec/OperatorSpec.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/task/RuntimeSystemContext.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/data/IncomingMessageTuple.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperator.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorFactoryImpl.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorSpec.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionOp.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionSpec.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/relation/Join.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/relation/JoinSpec.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/routing/SimpleRoutingContext.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStream.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStreamSpec.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/window/BoundedTimeWindow.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowSpec.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowState.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/task/RoutableRuntimeContext.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/task/StoredRuntimeContext.java PRE-CREATION 
>   samza-sql/src/test/java/org/apache/samza/sql/task/RandomOperatorTask.java PRE-CREATION 
>   samza-sql/src/test/java/org/apache/samza/sql/task/StreamSqlTask.java PRE-CREATION 
>   settings.gradle 3a01fd66359b8c79954ae8f34eeaf4b2e3fdc0b4 
> 
> Diff: https://reviews.apache.org/r/29754/diff/
> 
> 
> Testing
> -------
> 
> run ./bin/check-all.sh passed
> 
> 
> Thanks,
> 
> Yi Pan (Data Infrastructure)
> 
>


Re: Review Request 29754: StreamSQL operator API draft

Posted by Chris Riccomini <cr...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/29754/#review70380
-----------------------------------------------------------


Two high-level comments:

* RandomTask is complex. Developers will need recipes or good docs. This can be done as a separate. Just an observation.
* Too many "context" classes. In general, should re-evaluate how everything is named. I threw in some random suggestions for context renaming.

I'm still catching up on the discussion with Julian, so I might have more after that.


build.gradle
<https://reviews.apache.org/r/29754/#comment115519>

    Remove -d64 here. Tests should work on 32 bit machines.



samza-sql/src/main/java/org/apache/samza/sql/api/data/Tuple.java
<https://reviews.apache.org/r/29754/#comment115520>

    This API is a little strange to me. I'm assuming that this is getting a field from the message, right?



samza-sql/src/main/java/org/apache/samza/sql/api/operators/routing/OperatorRoutingContext.java
<https://reviews.apache.org/r/29754/#comment115523>

    OperatorRouter?



samza-sql/src/main/java/org/apache/samza/sql/api/task/RuntimeSystemContext.java
<https://reviews.apache.org/r/29754/#comment115521>

    SqlMessageCollector? RoutingMessageCollector?



samza-sql/src/main/java/org/apache/samza/sql/task/RoutableRuntimeContext.java
<https://reviews.apache.org/r/29754/#comment115526>

    OperatorMessageCollector?



samza-sql/src/main/java/org/apache/samza/sql/task/StoredRuntimeContext.java
<https://reviews.apache.org/r/29754/#comment115527>

    StoreMessageCollector?


- Chris Riccomini


On Jan. 29, 2015, 7:32 p.m., Yi Pan (Data Infrastructure) wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/29754/
> -----------------------------------------------------------
> 
> (Updated Jan. 29, 2015, 7:32 p.m.)
> 
> 
> Review request for samza, Chris Riccomini, Jay Kreps, Milinda Pathirage, Navina Ramesh, and Naveen Somasundaram.
> 
> 
> Bugs: SAMZA-482
>     https://issues.apache.org/jira/browse/SAMZA-482
> 
> 
> Repository: samza
> 
> 
> Description
> -------
> 
> Overview of the proposal:
> - 4 different types of operators implemented via 2 interface classes:
>    * stream-to-relation operators implementing TupleOperator interface and generate Relation output. E.g. Window operators
>    * stream-to-stream operators implementing TupleOperator interface and generate Tuple output. E.g. Partition operators
>    * relation-to-relation operators implementing RelationOperator interface and generate Relation output. E.g. All relational algebra operators, s.t. join, where, group-by, select, etc.
>    * relation-to-stream operators implementing RelationOperator interface and generate Tuple output. E.g. Istream or Dstream operators
> 
> Those operators are connected via the following two context interface classes:
> - RuntimeSystemContext which provides a context interface for the operators to send their output to
> - OperatorRoutingContext which provides the connection interface between the operators
> 
> In the example, we have enabled two execution models via the above two context classes:
> - RoutableRuntimeContext that uses the routing information from a OperatorRoutingContext and directly invoking the next operator when the current operator send its output via the RoutableRuntimeContext
> - StoredRuntimeContext that provides a storage for each operator's outputs that stored the output when the current operator send its output via StoredRuntimeContext. 
> 
> Then, it is up to the programmer to query the StoredRuntimeContext to get the operator's output and proceed w/ the next steps
> The first execution model allows the integration w/ future SQL parser and planner to automatically run a task, while the second model allows a random programmer to use the operators from the library in a random context.
>   
>   
> 2nd draft of SQL operator API
> - Simplified data object interface
>    * NOTE that the following two issues are still pending revision: i) nested data struct / schema; ii) multiple schema in a single stream?
> - Simplified operator constructor
>    * provided simple constructor of build-in operators w/o the need to use spec object and factory
> - Added runtime context to allow operators to send out their output
>    * this reduced the types of the operators to only two: RelationOperator and TupleOperator
> - Added optional routing context to connect operators to each other
> - Added two examples: 
>    * i) a fully specified application that generates operators via factory and automatically executes the operators via routing context
>    * ii) a "random" application that construct operators on-fly and use the runtime context to get the intermediate results back
> 
> 
> Diffs
> -----
> 
>   build.gradle 7a40ad4ae916610186848c06c4577e7067de98ee 
>   gradle/dependency-versions.gradle 44dd42603e93788562fd64c68312570cee71a2aa 
>   samza-sql/src/main/java/org/apache/samza/sql/api/data/EntityName.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/data/Relation.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/data/Tuple.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/Operator.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/RelationOperator.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/TupleOperator.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/routing/OperatorRoutingContext.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/operators/spec/OperatorSpec.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/api/task/RuntimeSystemContext.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/data/IncomingMessageTuple.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperator.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorFactoryImpl.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorSpec.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionOp.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionSpec.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/relation/Join.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/relation/JoinSpec.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/routing/SimpleRoutingContext.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStream.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStreamSpec.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/window/BoundedTimeWindow.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowSpec.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowState.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/task/RoutableRuntimeContext.java PRE-CREATION 
>   samza-sql/src/main/java/org/apache/samza/sql/task/StoredRuntimeContext.java PRE-CREATION 
>   samza-sql/src/test/java/org/apache/samza/sql/task/RandomOperatorTask.java PRE-CREATION 
>   samza-sql/src/test/java/org/apache/samza/sql/task/StreamSqlTask.java PRE-CREATION 
>   settings.gradle 3a01fd66359b8c79954ae8f34eeaf4b2e3fdc0b4 
> 
> Diff: https://reviews.apache.org/r/29754/diff/
> 
> 
> Testing
> -------
> 
> run ./bin/check-all.sh passed
> 
> 
> Thanks,
> 
> Yi Pan (Data Infrastructure)
> 
>


Re: Review Request 29754: StreamSQL operator API draft

Posted by "Yi Pan (Data Infrastructure)" <yi...@linkedin.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/29754/
-----------------------------------------------------------

(Updated Jan. 29, 2015, 7:32 p.m.)


Review request for samza, Chris Riccomini, Jay Kreps, Milinda Pathirage, Navina Ramesh, and Naveen Somasundaram.


Changes
-------

Fixed JDK6 compilation errors, added samza-kv as dependency


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


Repository: samza


Description
-------

Overview of the proposal:
- 4 different types of operators implemented via 2 interface classes:
   * stream-to-relation operators implementing TupleOperator interface and generate Relation output. E.g. Window operators
   * stream-to-stream operators implementing TupleOperator interface and generate Tuple output. E.g. Partition operators
   * relation-to-relation operators implementing RelationOperator interface and generate Relation output. E.g. All relational algebra operators, s.t. join, where, group-by, select, etc.
   * relation-to-stream operators implementing RelationOperator interface and generate Tuple output. E.g. Istream or Dstream operators

Those operators are connected via the following two context interface classes:
- RuntimeSystemContext which provides a context interface for the operators to send their output to
- OperatorRoutingContext which provides the connection interface between the operators

In the example, we have enabled two execution models via the above two context classes:
- RoutableRuntimeContext that uses the routing information from a OperatorRoutingContext and directly invoking the next operator when the current operator send its output via the RoutableRuntimeContext
- StoredRuntimeContext that provides a storage for each operator's outputs that stored the output when the current operator send its output via StoredRuntimeContext. 

Then, it is up to the programmer to query the StoredRuntimeContext to get the operator's output and proceed w/ the next steps
The first execution model allows the integration w/ future SQL parser and planner to automatically run a task, while the second model allows a random programmer to use the operators from the library in a random context.
  
  
2nd draft of SQL operator API
- Simplified data object interface
   * NOTE that the following two issues are still pending revision: i) nested data struct / schema; ii) multiple schema in a single stream?
- Simplified operator constructor
   * provided simple constructor of build-in operators w/o the need to use spec object and factory
- Added runtime context to allow operators to send out their output
   * this reduced the types of the operators to only two: RelationOperator and TupleOperator
- Added optional routing context to connect operators to each other
- Added two examples: 
   * i) a fully specified application that generates operators via factory and automatically executes the operators via routing context
   * ii) a "random" application that construct operators on-fly and use the runtime context to get the intermediate results back


Diffs (updated)
-----

  build.gradle 7a40ad4ae916610186848c06c4577e7067de98ee 
  gradle/dependency-versions.gradle 44dd42603e93788562fd64c68312570cee71a2aa 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/EntityName.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/Relation.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/Tuple.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/Operator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/RelationOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/TupleOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/routing/OperatorRoutingContext.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/spec/OperatorSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/task/RuntimeSystemContext.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/data/IncomingMessageTuple.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorFactoryImpl.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionOp.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/relation/Join.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/relation/JoinSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/routing/SimpleRoutingContext.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStream.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStreamSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/BoundedTimeWindow.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowState.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/task/RoutableRuntimeContext.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/task/StoredRuntimeContext.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/task/RandomOperatorTask.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/task/StreamSqlTask.java PRE-CREATION 
  settings.gradle 3a01fd66359b8c79954ae8f34eeaf4b2e3fdc0b4 

Diff: https://reviews.apache.org/r/29754/diff/


Testing (updated)
-------

run ./bin/check-all.sh passed


Thanks,

Yi Pan (Data Infrastructure)


Re: Review Request 29754: StreamSQL operator API draft

Posted by "Yi Pan (Data Infrastructure)" <yi...@linkedin.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/29754/
-----------------------------------------------------------

(Updated Jan. 22, 2015, 11:41 p.m.)


Review request for samza, Chris Riccomini, Jay Kreps, Milinda Pathirage, Navina Ramesh, and Naveen Somasundaram.


Changes
-------

Accommodated most of Chris's comments including: consolidate the classes with existing Samza classes, add multiple outputs support for the operators.


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


Repository: samza


Description
-------

Overview of the proposal:
- 4 different types of operators implemented via 2 interface classes:
   * stream-to-relation operators implementing TupleOperator interface and generate Relation output. E.g. Window operators
   * stream-to-stream operators implementing TupleOperator interface and generate Tuple output. E.g. Partition operators
   * relation-to-relation operators implementing RelationOperator interface and generate Relation output. E.g. All relational algebra operators, s.t. join, where, group-by, select, etc.
   * relation-to-stream operators implementing RelationOperator interface and generate Tuple output. E.g. Istream or Dstream operators

Those operators are connected via the following two context interface classes:
- RuntimeSystemContext which provides a context interface for the operators to send their output to
- OperatorRoutingContext which provides the connection interface between the operators

In the example, we have enabled two execution models via the above two context classes:
- RoutableRuntimeContext that uses the routing information from a OperatorRoutingContext and directly invoking the next operator when the current operator send its output via the RoutableRuntimeContext
- StoredRuntimeContext that provides a storage for each operator's outputs that stored the output when the current operator send its output via StoredRuntimeContext. 

Then, it is up to the programmer to query the StoredRuntimeContext to get the operator's output and proceed w/ the next steps
The first execution model allows the integration w/ future SQL parser and planner to automatically run a task, while the second model allows a random programmer to use the operators from the library in a random context.
  
  
2nd draft of SQL operator API
- Simplified data object interface
   * NOTE that the following two issues are still pending revision: i) nested data struct / schema; ii) multiple schema in a single stream?
- Simplified operator constructor
   * provided simple constructor of build-in operators w/o the need to use spec object and factory
- Added runtime context to allow operators to send out their output
   * this reduced the types of the operators to only two: RelationOperator and TupleOperator
- Added optional routing context to connect operators to each other
- Added two examples: 
   * i) a fully specified application that generates operators via factory and automatically executes the operators via routing context
   * ii) a "random" application that construct operators on-fly and use the runtime context to get the intermediate results back


Diffs (updated)
-----

  build.gradle 7a40ad4ae916610186848c06c4577e7067de98ee 
  gradle/dependency-versions.gradle 44dd42603e93788562fd64c68312570cee71a2aa 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/EntityName.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/Relation.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/Tuple.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/Operator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/RelationOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/TupleOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/routing/OperatorRoutingContext.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/spec/OperatorSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/task/RuntimeSystemContext.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/data/IncomingMessageTuple.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorFactoryImpl.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionOp.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/relation/Join.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/relation/JoinSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/routing/SimpleRoutingContext.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStream.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStreamSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/BoundedTimeWindow.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowState.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/task/RandomOperatorTask.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/task/RoutableRuntimeContext.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/task/StoredRuntimeContext.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/task/StreamSqlTask.java PRE-CREATION 
  settings.gradle 3a01fd66359b8c79954ae8f34eeaf4b2e3fdc0b4 

Diff: https://reviews.apache.org/r/29754/diff/


Testing
-------


Thanks,

Yi Pan (Data Infrastructure)


Re: Review Request 29754: StreamSQL operator API draft

Posted by "Yi Pan (Data Infrastructure)" <yi...@linkedin.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/29754/
-----------------------------------------------------------

(Updated Jan. 13, 2015, 12:56 a.m.)


Review request for samza, Chris Riccomini, Jay Kreps, Milinda Pathirage, Navina Ramesh, and Naveen Somasundaram.


Changes
-------

Updated the overview of the proposed APIs


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


Repository: samza


Description (updated)
-------

Overview of the proposal:
- 4 different types of operators implemented via 2 interface classes:
   * stream-to-relation operators implementing TupleOperator interface and generate Relation output. E.g. Window operators
   * stream-to-stream operators implementing TupleOperator interface and generate Tuple output. E.g. Partition operators
   * relation-to-relation operators implementing RelationOperator interface and generate Relation output. E.g. All relational algebra operators, s.t. join, where, group-by, select, etc.
   * relation-to-stream operators implementing RelationOperator interface and generate Tuple output. E.g. Istream or Dstream operators

Those operators are connected via the following two context interface classes:
- RuntimeSystemContext which provides a context interface for the operators to send their output to
- OperatorRoutingContext which provides the connection interface between the operators

In the example, we have enabled two execution models via the above two context classes:
- RoutableRuntimeContext that uses the routing information from a OperatorRoutingContext and directly invoking the next operator when the current operator send its output via the RoutableRuntimeContext
- StoredRuntimeContext that provides a storage for each operator's outputs that stored the output when the current operator send its output via StoredRuntimeContext. 

Then, it is up to the programmer to query the StoredRuntimeContext to get the operator's output and proceed w/ the next steps
The first execution model allows the integration w/ future SQL parser and planner to automatically run a task, while the second model allows a random programmer to use the operators from the library in a random context.
  
  
2nd draft of SQL operator API
- Simplified data object interface
   * NOTE that the following two issues are still pending revision: i) nested data struct / schema; ii) multiple schema in a single stream?
- Simplified operator constructor
   * provided simple constructor of build-in operators w/o the need to use spec object and factory
- Added runtime context to allow operators to send out their output
   * this reduced the types of the operators to only two: RelationOperator and TupleOperator
- Added optional routing context to connect operators to each other
- Added two examples: 
   * i) a fully specified application that generates operators via factory and automatically executes the operators via routing context
   * ii) a "random" application that construct operators on-fly and use the runtime context to get the intermediate results back


Diffs
-----

  build.gradle 38383bd9e3f0847d6088a4ea4c1ee6f3dcd1e430 
  gradle/dependency-versions.gradle 44dd42603e93788562fd64c68312570cee71a2aa 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/IncomingMessageTuple.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/OutgoingMessageTuple.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/Relation.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/Tuple.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/Operator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/RelationOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/TupleOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/routing/OperatorRoutingContext.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/spec/OperatorSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/task/InitSystemContext.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/task/RuntimeSystemContext.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorFactoryImpl.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionOp.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/relation/Join.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/relation/JoinSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStream.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStreamSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/BoundedTimeWindow.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowState.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/data/SystemInputTuple.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/operators/routing/SimpleRoutingContext.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/store/SqlContextManager.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/task/RandomOperatorTask.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/task/RoutableRuntimeContext.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/task/StoredRuntimeContext.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/task/StreamSqlTask.java PRE-CREATION 
  settings.gradle 3a01fd66359b8c79954ae8f34eeaf4b2e3fdc0b4 

Diff: https://reviews.apache.org/r/29754/diff/


Testing
-------


Thanks,

Yi Pan (Data Infrastructure)


Re: Review Request 29754: StreamSQL operator API draft

Posted by "Yi Pan (Data Infrastructure)" <yi...@linkedin.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/29754/
-----------------------------------------------------------

(Updated Jan. 13, 2015, 12:32 a.m.)


Review request for samza, Chris Riccomini, Jay Kreps, Milinda Pathirage, Navina Ramesh, and Naveen Somasundaram.


Changes
-------

Updated by adding proper javadoc and Apache licenses to the diff.


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


Repository: samza


Description
-------

2nd draft of SQL operator API
- Simplified data object interface
   * NOTE that the following two issues are still pending revision: i) nested data struct / schema; ii) multiple schema in a single stream?
- Simplified operator constructor
   * provided simple constructor of build-in operators w/o the need to use spec object and factory
- Added runtime context to allow operators to send out their output
   * this reduced the types of the operators to only two: RelationOperator and TupleOperator
- Added optional routing context to connect operators to each other
- Added two examples: 
   * i) a fully specified application that generates operators via factory and automatically executes the operators via routing context
   * ii) a "random" application that construct operators on-fly and use the runtime context to get the intermediate results back

NOTE: there is still cleanups to be done to add java docs to the code and there will be more updates coming.
  


Diffs (updated)
-----

  build.gradle 38383bd9e3f0847d6088a4ea4c1ee6f3dcd1e430 
  gradle/dependency-versions.gradle 44dd42603e93788562fd64c68312570cee71a2aa 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/IncomingMessageTuple.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/OutgoingMessageTuple.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/Relation.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/Tuple.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/Operator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/RelationOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/TupleOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/routing/OperatorRoutingContext.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/spec/OperatorSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/task/InitSystemContext.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/task/RuntimeSystemContext.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorFactoryImpl.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionOp.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/relation/Join.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/relation/JoinSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStream.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStreamSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/BoundedTimeWindow.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowState.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/data/SystemInputTuple.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/operators/routing/SimpleRoutingContext.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/store/SqlContextManager.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/task/RandomOperatorTask.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/task/RoutableRuntimeContext.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/task/StoredRuntimeContext.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/task/StreamSqlTask.java PRE-CREATION 
  settings.gradle 3a01fd66359b8c79954ae8f34eeaf4b2e3fdc0b4 

Diff: https://reviews.apache.org/r/29754/diff/


Testing
-------


Thanks,

Yi Pan (Data Infrastructure)


Re: Review Request 29754: StreamSQL operator API draft

Posted by "Yi Pan (Data Infrastructure)" <yi...@linkedin.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/29754/
-----------------------------------------------------------

(Updated Jan. 10, 2015, 12:06 a.m.)


Review request for samza, Chris Riccomini, Jay Kreps, Milinda Pathirage, Navina Ramesh, and Naveen Somasundaram.


Summary (updated)
-----------------

StreamSQL operator API draft


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


Repository: samza


Description
-------

2nd draft of SQL operator API
- Simplified data object interface
   * NOTE that the following two issues are still pending revision: i) nested data struct / schema; ii) multiple schema in a single stream?
- Simplified operator constructor
   * provided simple constructor of build-in operators w/o the need to use spec object and factory
- Added runtime context to allow operators to send out their output
   * this reduced the types of the operators to only two: RelationOperator and TupleOperator
- Added optional routing context to connect operators to each other
- Added two examples: 
   * i) a fully specified application that generates operators via factory and automatically executes the operators via routing context
   * ii) a "random" application that construct operators on-fly and use the runtime context to get the intermediate results back

NOTE: there is still cleanups to be done to add java docs to the code and there will be more updates coming.
  


Diffs
-----

  build.gradle 38383bd9e3f0847d6088a4ea4c1ee6f3dcd1e430 
  gradle/dependency-versions.gradle 44dd42603e93788562fd64c68312570cee71a2aa 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/IncomingMessageTuple.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/OutgoingMessageTuple.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/Relation.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/Tuple.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/Operator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/RelationOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/TupleOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/routing/OperatorRoutingContext.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/spec/OperatorSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/task/InitSystemContext.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/task/RuntimeSystemContext.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorFactoryImpl.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionOp.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/relation/Join.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/relation/JoinSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStream.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStreamSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/BoundedTimeWindow.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowState.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/data/SystemInputTuple.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/operators/routing/SimpleRoutingContext.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/store/SqlContextManager.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/task/RandomOperatorTask.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/task/RoutableRuntimeContext.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/task/StoredRuntimeContext.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/task/StreamSqlTask.java PRE-CREATION 
  settings.gradle 3a01fd66359b8c79954ae8f34eeaf4b2e3fdc0b4 

Diff: https://reviews.apache.org/r/29754/diff/


Testing
-------


Thanks,

Yi Pan (Data Infrastructure)


Re: Review Request 29754: StreamSQL operator API draft - This is the first draft of the StreamSQL operator APIs - org.apache.samza.sql.api.* contains definitions of all interface classes - org.apache.samza.sql.operators.

Posted by "Yi Pan (Data Infrastructure)" <yi...@linkedin.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/29754/
-----------------------------------------------------------

(Updated Jan. 9, 2015, 11:11 p.m.)


Review request for samza, Chris Riccomini, Jay Kreps, Milinda Pathirage, Navina Ramesh, and Naveen Somasundaram.


Changes
-------

A 2nd draft for SQL operator APIs


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


Repository: samza


Description
-------

2nd draft of SQL operator API
- Simplified data object interface
   * NOTE that the following two issues are still pending revision: i) nested data struct / schema; ii) multiple schema in a single stream?
- Simplified operator constructor
   * provided simple constructor of build-in operators w/o the need to use spec object and factory
- Added runtime context to allow operators to send out their output
   * this reduced the types of the operators to only two: RelationOperator and TupleOperator
- Added optional routing context to connect operators to each other
- Added two examples: 
   * i) a fully specified application that generates operators via factory and automatically executes the operators via routing context
   * ii) a "random" application that construct operators on-fly and use the runtime context to get the intermediate results back

NOTE: there is still cleanups to be done to add java docs to the code and there will be more updates coming.
  


Diffs
-----

  build.gradle 38383bd9e3f0847d6088a4ea4c1ee6f3dcd1e430 
  gradle/dependency-versions.gradle 44dd42603e93788562fd64c68312570cee71a2aa 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/IncomingMessageTuple.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/OutgoingMessageTuple.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/Relation.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/Tuple.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/Operator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/RelationOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/TupleOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/routing/OperatorRoutingContext.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/spec/OperatorSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/task/InitSystemContext.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/task/RuntimeSystemContext.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorFactoryImpl.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionOp.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/relation/Join.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/relation/JoinSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStream.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStreamSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/BoundedTimeWindow.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowState.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/data/SystemInputTuple.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/operators/routing/SimpleRoutingContext.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/store/SqlContextManager.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/task/RandomOperatorTask.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/task/RoutableRuntimeContext.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/task/StoredRuntimeContext.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/task/StreamSqlTask.java PRE-CREATION 
  settings.gradle 3a01fd66359b8c79954ae8f34eeaf4b2e3fdc0b4 

Diff: https://reviews.apache.org/r/29754/diff/


Testing
-------


Thanks,

Yi Pan (Data Infrastructure)


Re: Review Request 29754: StreamSQL operator API draft - This is the first draft of the StreamSQL operator APIs - org.apache.samza.sql.api.* contains definitions of all interface classes - org.apache.samza.sql.operators.

Posted by "Yi Pan (Data Infrastructure)" <yi...@linkedin.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/29754/
-----------------------------------------------------------

(Updated Jan. 9, 2015, 11:10 p.m.)


Review request for samza, Chris Riccomini, Jay Kreps, Milinda Pathirage, Navina Ramesh, and Naveen Somasundaram.


Repository: samza


Description (updated)
-------

2nd draft of SQL operator API
- Simplified data object interface
   * NOTE that the following two issues are still pending revision: i) nested data struct / schema; ii) multiple schema in a single stream?
- Simplified operator constructor
   * provided simple constructor of build-in operators w/o the need to use spec object and factory
- Added runtime context to allow operators to send out their output
   * this reduced the types of the operators to only two: RelationOperator and TupleOperator
- Added optional routing context to connect operators to each other
- Added two examples: 
   * i) a fully specified application that generates operators via factory and automatically executes the operators via routing context
   * ii) a "random" application that construct operators on-fly and use the runtime context to get the intermediate results back

NOTE: there is still cleanups to be done to add java docs to the code and there will be more updates coming.
  


Diffs
-----

  build.gradle 38383bd9e3f0847d6088a4ea4c1ee6f3dcd1e430 
  gradle/dependency-versions.gradle 44dd42603e93788562fd64c68312570cee71a2aa 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/IncomingMessageTuple.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/OutgoingMessageTuple.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/Relation.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/Tuple.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/Operator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/RelationOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/TupleOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/routing/OperatorRoutingContext.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/spec/OperatorSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/task/InitSystemContext.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/task/RuntimeSystemContext.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorFactoryImpl.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionOp.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/relation/Join.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/relation/JoinSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStream.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStreamSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/BoundedTimeWindow.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowState.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/data/SystemInputTuple.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/operators/routing/SimpleRoutingContext.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/store/SqlContextManager.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/task/RandomOperatorTask.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/task/RoutableRuntimeContext.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/task/StoredRuntimeContext.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/task/StreamSqlTask.java PRE-CREATION 
  settings.gradle 3a01fd66359b8c79954ae8f34eeaf4b2e3fdc0b4 

Diff: https://reviews.apache.org/r/29754/diff/


Testing
-------


Thanks,

Yi Pan (Data Infrastructure)


Re: Review Request 29754: StreamSQL operator API draft - This is the first draft of the StreamSQL operator APIs - org.apache.samza.sql.api.* contains definitions of all interface classes - org.apache.samza.sql.operators.

Posted by "Yi Pan (Data Infrastructure)" <yi...@linkedin.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/29754/
-----------------------------------------------------------

(Updated Jan. 9, 2015, 11:07 p.m.)


Review request for samza, Chris Riccomini, Navina Ramesh, and Naveen Somasundaram.


Repository: samza


Description
-------

2nd draft of SQL operator API
- Simplified data object interface
   * NOTE that the following two issues are still pending revision: i) nested data struct / schema; ii) multiple schema in a single stream?
- Simplified operator constructor
   * provided simple constructor of build-in operators w/o the need to use spec object and factory
- Added runtime context to allow operators to send out their output
   * this reduced the types of the operators to only two: RelationOperator and TupleOperator
- Added optional routing context to connect operators to each other
- Added two examples: 
   * i) a fully specified application that generates operators via factory and automatically executes the operators via routing context
   * ii) a "random" application that construct operators on-fly and use the runtime context to get the intermediate results back


Diffs
-----

  build.gradle 38383bd9e3f0847d6088a4ea4c1ee6f3dcd1e430 
  gradle/dependency-versions.gradle 44dd42603e93788562fd64c68312570cee71a2aa 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/IncomingMessageTuple.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/OutgoingMessageTuple.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/Relation.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/data/Tuple.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/Operator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/RelationOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/SqlOperatorFactory.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/TupleOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/routing/OperatorRoutingContext.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/operators/spec/OperatorSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/task/InitSystemContext.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/api/task/RuntimeSystemContext.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperator.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/factory/SimpleOperatorFactoryImpl.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionOp.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/partition/PartitionSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/relation/Join.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/relation/JoinSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStream.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/stream/InsertStreamSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/BoundedTimeWindow.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowSpec.java PRE-CREATION 
  samza-sql/src/main/java/org/apache/samza/sql/operators/window/WindowState.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/data/SystemInputTuple.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/operators/routing/SimpleRoutingContext.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/store/SqlContextManager.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/task/RandomOperatorTask.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/task/RoutableRuntimeContext.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/task/StoredRuntimeContext.java PRE-CREATION 
  samza-sql/src/test/java/org/apache/samza/sql/task/StreamSqlTask.java PRE-CREATION 
  settings.gradle 3a01fd66359b8c79954ae8f34eeaf4b2e3fdc0b4 

Diff: https://reviews.apache.org/r/29754/diff/


Testing
-------


Thanks,

Yi Pan (Data Infrastructure)