> 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)
> 
>

Reply via email to