[ 
https://issues.apache.org/jira/browse/SAMZA-482?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14268275#comment-14268275
 ] 

Yi Pan (Data Infrastructure) commented on SAMZA-482:
----------------------------------------------------

Re-typing what I lost yesterday. :(

[~criccomini], thanks for the great summary on the discussions on the first 
draft of operator API! I will try to focus on 2) and 3) first.

{quote}
2. Is the intent of the API to make it usable for developers, or just as an 
implementation detail of SQL/DSLs?
Regarding (2), an example would be a developer that just wants to use the 
"Join" operator inside a StreamTask, and then put some custom logic before and 
after the join. Qualitatively, the API seems a bit cumbersome for a random 
developer to use. I think part of the complexity might come from using specs, 
rather than just directly passing parameters into methods. Another thought here 
is how the developer might get the messages back out. It seems like they'd have 
to write a custom operator that buffered the messages, so they could be 
retrieved.
{quote}

I agree that the overhead is probably coming from the spec classes. The sole 
purpose of the spec classes is try to encapsulate the details of the 
configuration/specification of each operator s.t. the factory class can have a 
unified interface to create all different types of operators. It also 
simplifies the constructor of operator since it can have the same function 
signature while the specification of the operator evolves over time. For random 
developers that does not want to overhead of spec classes, maybe we can provide 
simpler version of constructors on top of the build-in operators, to which the 
user can directly passing the parameters to the constructor to instantiate a 
simple operator. I will add application examples illustrating how a random 
developer can use this simpler way of instantiating the operators s.t. we can 
see how it works.

{quote}
3. The proposed API uses operators that are aware of each other. The routing 
happens within the operators (this.nextOp.process(tuple)). The alternative is 
to have the routing happen outside of the operators.
Question (3) seems related to the mutable setter methods as well. If routing is 
handled outside of the operators, it seems that the operators could be much 
more immutable, since they no longer need setter methods other than init and 
process. One trade off here would be that operators that accrue large outgoing 
message buffers within a single call might run out of memory, since the routing 
logic doesn't have a chance to run until the operator returns (this is 
essentially the equivalent of what we had with buffering messages in 
MessageCollector, before we switched to immediately sending messages when 
collector.send is called). Perhaps there are work arounds that would make this 
approach viable, though. I'll have to think about it.
{quote}

In deed, the issue raised here is the root cause of the complexity in the 
different types of operators, since the current implementation requires the 
operators to be aware of the next op s.t. its own output matches the next op's 
input. I have taken down the path of striping the routing part out of the 
operator class and create a operator routing context to handle the connection 
between operators. I personally liked this version better due to the way it 
simplified the operator classes and removal of the mutable setters. There are 
still two methods to control the execution of the operators, as [~criccomini] 
mentioned above: 1) calling the operator.process() / timeout() and let the 
operator invoke the next operator via the routing context; Or 2) calling the 
operator.process() / timeout() directly from the outside routing context and 
returns w/o invoking the next operator. I chose 1) in the implementation due to 
the following reason: in most of cases, the logic that determines whether the 
input values should trigger an output to be sent to the next operator is 
decided by the operator's internal state and logic. Moving this part of logic 
outside the operator class seem to be a bit odd to me. Hence, I chose the 
solution to break up the routing to the next operator and the decision on 
whether to trigger the next operator into two parts: i) finding the next 
operator and invoke it is done via the routing context; ii) changing the 
internal state and making the decision whether to send the output via the 
routing context is done by the operator. Hence, the operator's process() and 
timeout() functions will take one additional parameter: the routing context 
object. For example, when the operator decides that it should send the output 
to the next relation operator, it simple calls 
OperatorRoutingContext.sendToNextRelationOperator(currentOp, deltaRelation). 
The routing context object actually implements the routing part to find the 
next operator and invoke the process() method. Method 2) has the following 
advantages to a random developer: i) the control to whether to invoke the next 
operators or not is immediately coming back to the programmer when process() 
completes; ii) after the process() returns, the output of the current 
invocation of the operator can be retrieved immediately. I think that we can 
achieve the same via 1) by overriding the method in 
OperatorRoutingContext.sendToXXX() method by just recording the output, w/ 
invoking the next operator. Doing that, we can: i) provide the full control to 
the random developer on return of operator.process() without invoking the next 
operator; ii) provide output from an operator via the routing context after the 
return from process(). I will experiment more on this and post another review 
board request on this.

Thanks!


> Identify the set of operators for SQL on Samza
> ----------------------------------------------
>
>                 Key: SAMZA-482
>                 URL: https://issues.apache.org/jira/browse/SAMZA-482
>             Project: Samza
>          Issue Type: Sub-task
>            Reporter: Yi Pan (Data Infrastructure)
>            Priority: Minor
>              Labels: project
>
> This came out of a discussion between [~milinda], [~criccomini], and 
> [~nickpan47]. We think that it will be a good idea to separate the operators 
> layer from the high-level language layer, s.t. we can allow different 
> languages to be built on-top-of the same set of fundamental functions (i.e. 
> SQL-like or DSL).



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to