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

Chris Riccomini commented on SAMZA-482:
---------------------------------------

I left some comments on the RB. Some high-level thoughts:

# There is no DDL or concept of a schema in the proposed API.
# Is the intent of the API to make it usable for developers, or just as an 
implementation detail of SQL/DSLs?
# 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.
# How does tuple work with nested data structures (maps, lists, etc)? We use 
Samza with Avro, and these schemas can have nested fields. It would be nice to 
be able to query them.

Regarding (1), it's definitely simpler to exclude the DDL if we don't need it, 
but I wonder if we do. It seems likely that we would need it for some things. 
We should think through this part. And if we do need schemas, should they be at 
the stream-level or tuple-level (i.e. can you have tuples with different 
schemas in the same stream?).

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.

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.

Regarding (4), Hive seems to have [a nice 
concept|http://blog.cloudera.com/blog/2012/11/analyzing-twitter-data-with-hadoop-part-3-querying-semi-structured-data-with-hive/]
 for nested structures.

> 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