Hi All, I have attached a patch to SAMZA-561 ( https://issues.apache.org/jira/browse/SAMZA-561) which demonstrate streaming SQL execution planning functionality. Attached patch only supports stream filtering and ‘insert into’ for sending the filtered stream to a some other topic. This patch also comes with a integration test which demonstrate and validate the stream filtering capability. Below are some facts about the current implementation.
- Streaming SQL is packaged into org.apache.samza.task.sql.StreamSqlTask which is a Samza StreamTask. - Stream SQL query, Avro schema (used to initialize SQL serde for Avro) and Calcite model are configured as StreamSqlTask properties. (Please refer SAMZA_SRC/samza-test/src/main/config/sql-filter.properties) - Avro schema and Calcite model is available in the class path and special URL format (e.g. resource:orders.avsc) is used to refer to them. (Note: This should be changed.) - Whole query get executed inside the same Samza task - Projections are not there yet. My next goal is to implement basic aggregations and projections. Projections can change the schema of the final output. We need to discuss whether we need user to specify the output schema or we handle (track) the changes to the schema in the query layer (still not sure whether this is possible or not). According to my current understanding about Calcite, its possible to get type of a row after a projection or any other operation. I believe generating the output schema based on this row type is possible. I will provide more details once I started to working on projections. Also, please feel free to comment about how I have implemented the StreamSqlTask. I would like to know whether this approach to configuring and scheduling streaming queries is okay. Also any other comments on improvements are welcome. Thanks Milinda -- Milinda Pathirage PhD Student | Research Assistant School of Informatics and Computing | Data to Insight Center Indiana University twitter: milindalakmal skype: milinda.pathirage blog: http://milinda.pathirage.org