@@ -2,12 +2,30 @@ use crate::config::injection::DIService;
22use crate :: config:: ConfigObj ;
33use crate :: metastore:: table:: StreamOffset ;
44use crate :: metastore:: Column ;
5+ use crate :: sql:: MySqlDialectWithBackTicks ;
56use crate :: streaming:: { parse_json_payload_and_key, StreamingSource } ;
67use crate :: table:: { Row , TableValue } ;
78use crate :: CubeError ;
9+ use arrow:: array:: ArrayRef ;
10+ use arrow:: record_batch:: RecordBatch ;
11+ use arrow:: { datatypes:: Schema , datatypes:: SchemaRef } ;
812use async_std:: stream;
913use async_trait:: async_trait;
14+ use datafusion:: catalog:: TableReference ;
1015use datafusion:: cube_ext;
16+ use datafusion:: datasource:: datasource:: Statistics ;
17+ use datafusion:: datasource:: TableProvider ;
18+ use datafusion:: error:: DataFusionError ;
19+ use datafusion:: logical_plan:: Expr as DExpr ;
20+ use datafusion:: logical_plan:: LogicalPlan ;
21+ use datafusion:: physical_plan:: empty:: EmptyExec ;
22+ use datafusion:: physical_plan:: memory:: MemoryExec ;
23+ use datafusion:: physical_plan:: udaf:: AggregateUDF ;
24+ use datafusion:: physical_plan:: udf:: ScalarUDF ;
25+ use datafusion:: physical_plan:: { collect, ExecutionPlan } ;
26+ use datafusion:: prelude:: ExecutionContext ;
27+ use datafusion:: sql:: parser:: Statement as DFStatement ;
28+ use datafusion:: sql:: planner:: { ContextProvider , SqlToRel } ;
1129use futures:: Stream ;
1230use json:: object:: Object ;
1331use json:: JsonValue ;
@@ -16,6 +34,10 @@ use rdkafka::error::KafkaResult;
1634use rdkafka:: message:: BorrowedMessage ;
1735use rdkafka:: util:: Timeout ;
1836use rdkafka:: { ClientConfig , Message , Offset , TopicPartitionList } ;
37+ use sqlparser:: ast:: { Query , SetExpr , Statement } ;
38+ use sqlparser:: parser:: Parser ;
39+ use sqlparser:: tokenizer:: Tokenizer ;
40+ use std:: any:: Any ;
1941use std:: pin:: Pin ;
2042use std:: sync:: Arc ;
2143use std:: time:: Duration ;
@@ -28,18 +50,18 @@ pub struct KafkaStreamingSource {
2850 password : Option < String > ,
2951 topic : String ,
3052 host : String ,
31- // TODO Support parsing of filters and applying before insert
32- _select_statement : Option < String > ,
3353 offset : Option < StreamOffset > ,
3454 partition : usize ,
3555 kafka_client : Arc < dyn KafkaClientService > ,
3656 use_ssl : bool ,
57+ post_filter : Option < Arc < dyn ExecutionPlan > > ,
3758}
3859
3960impl KafkaStreamingSource {
4061 pub fn new (
4162 table_id : u64 ,
4263 unique_key_columns : Vec < Column > ,
64+ columns : Vec < Column > ,
4365 user : Option < String > ,
4466 password : Option < String > ,
4567 topic : String ,
@@ -50,18 +72,156 @@ impl KafkaStreamingSource {
5072 kafka_client : Arc < dyn KafkaClientService > ,
5173 use_ssl : bool ,
5274 ) -> Self {
75+ let post_filter = if let Some ( select_statement) = select_statement {
76+ let planner = KafkaFilterPlanner {
77+ topic : topic. clone ( ) ,
78+ columns,
79+ } ;
80+ match planner. parse_select_statement ( select_statement. clone ( ) ) {
81+ Ok ( p) => p,
82+ Err ( e) => {
83+ //FIXME May be we should stop execution here
84+ log:: error!(
85+ "Error while parsing `select_statement`: {}. Select statement ignored" ,
86+ e
87+ ) ;
88+ None
89+ }
90+ }
91+ } else {
92+ None
93+ } ;
94+
5395 KafkaStreamingSource {
5496 table_id,
5597 unique_key_columns,
5698 user,
5799 password,
58100 topic,
59101 host,
60- _select_statement : select_statement,
61102 offset,
62103 partition,
63104 kafka_client,
64105 use_ssl,
106+ post_filter,
107+ }
108+ }
109+ }
110+
111+ pub struct KafkaFilterPlanner {
112+ topic : String ,
113+ columns : Vec < Column > ,
114+ }
115+
116+ impl KafkaFilterPlanner {
117+ fn parse_select_statement (
118+ & self ,
119+ select_statement : String ,
120+ ) -> Result < Option < Arc < dyn ExecutionPlan > > , CubeError > {
121+ let dialect = & MySqlDialectWithBackTicks { } ;
122+ let mut tokenizer = Tokenizer :: new ( dialect, & select_statement) ;
123+ let tokens = tokenizer. tokenize ( ) . unwrap ( ) ;
124+ let statement = Parser :: new ( tokens, dialect) . parse_statement ( ) ?;
125+
126+ match & statement {
127+ Statement :: Query ( box Query {
128+ body : SetExpr :: Select ( s) ,
129+ ..
130+ } ) => {
131+ if s. selection . is_none ( ) {
132+ return Ok ( None ) ;
133+ }
134+ let provider = TopicTableProvider :: new ( self . topic . clone ( ) , & self . columns ) ;
135+ let query_planner = SqlToRel :: new ( & provider) ;
136+ let logical_plan =
137+ query_planner. statement_to_plan ( & DFStatement :: Statement ( statement. clone ( ) ) ) ?;
138+ let physical_filter = Self :: make_physical_filter ( & logical_plan) ?;
139+ Ok ( physical_filter)
140+ }
141+ _ => Err ( CubeError :: user ( format ! (
142+ "{} is not valid select query" ,
143+ select_statement
144+ ) ) ) ,
145+ }
146+ }
147+
148+ /// Only Projection > Filter > TableScan plans are allowed
149+ fn make_physical_filter (
150+ plan : & LogicalPlan ,
151+ ) -> Result < Option < Arc < dyn ExecutionPlan > > , CubeError > {
152+ match plan {
153+ LogicalPlan :: Projection { input, .. } => match input. as_ref ( ) {
154+ filter_plan @ LogicalPlan :: Filter { input, .. } => match input. as_ref ( ) {
155+ LogicalPlan :: TableScan { .. } => {
156+ let plan_ctx = Arc :: new ( ExecutionContext :: new ( ) ) ;
157+ let phys_plan = plan_ctx. create_physical_plan ( & filter_plan) ?;
158+ Ok ( Some ( phys_plan) )
159+ }
160+ _ => Ok ( None ) ,
161+ } ,
162+ _ => Ok ( None ) ,
163+ } ,
164+ _ => Ok ( None ) ,
165+ }
166+ }
167+ }
168+
169+ #[ derive( Debug , Clone ) ]
170+ struct TopicTableProvider {
171+ topic : String ,
172+ schema : SchemaRef ,
173+ }
174+
175+ impl TopicTableProvider {
176+ pub fn new ( topic : String , columns : & Vec < Column > ) -> Self {
177+ let schema = Arc :: new ( Schema :: new (
178+ columns. iter ( ) . map ( |c| c. clone ( ) . into ( ) ) . collect :: < Vec < _ > > ( ) ,
179+ ) ) ;
180+ Self { topic, schema }
181+ }
182+ }
183+
184+ impl ContextProvider for TopicTableProvider {
185+ fn get_table_provider ( & self , name : TableReference ) -> Option < Arc < dyn TableProvider > > {
186+ match name {
187+ TableReference :: Bare { table } if table == self . topic => Some ( Arc :: new ( self . clone ( ) ) ) ,
188+ _ => None ,
189+ }
190+ }
191+
192+ fn get_function_meta ( & self , _name : & str ) -> Option < Arc < ScalarUDF > > {
193+ None
194+ }
195+
196+ fn get_aggregate_meta ( & self , _name : & str ) -> Option < Arc < AggregateUDF > > {
197+ None
198+ }
199+ }
200+
201+ impl TableProvider for TopicTableProvider {
202+ fn as_any ( & self ) -> & dyn Any {
203+ self
204+ }
205+
206+ fn schema ( & self ) -> SchemaRef {
207+ self . schema . clone ( )
208+ }
209+
210+ fn scan (
211+ & self ,
212+ _projection : & Option < Vec < usize > > ,
213+ _batch_size : usize ,
214+ _filters : & [ DExpr ] ,
215+ _limit : Option < usize > ,
216+ ) -> Result < Arc < dyn ExecutionPlan > , DataFusionError > {
217+ Ok ( Arc :: new ( EmptyExec :: new ( false , self . schema ( ) ) ) )
218+ }
219+
220+ fn statistics ( & self ) -> Statistics {
221+ Statistics {
222+ num_rows : None ,
223+ total_byte_size : None ,
224+ column_statistics : None ,
65225 }
66226 }
67227}
@@ -297,6 +457,25 @@ impl StreamingSource for KafkaStreamingSource {
297457 Ok ( stream)
298458 }
299459
460+ async fn apply_post_filter ( & self , data : Vec < ArrayRef > ) -> Result < Vec < ArrayRef > , CubeError > {
461+ if let Some ( post_filter) = & self . post_filter {
462+ let schema = post_filter. children ( ) [ 0 ] . schema ( ) ;
463+ let batch = RecordBatch :: try_new ( schema. clone ( ) , data) ?;
464+ let input = Arc :: new ( MemoryExec :: try_new ( & [ vec ! [ batch] ] , schema. clone ( ) , None ) ?) ;
465+ let filter = post_filter. with_new_children ( vec ! [ input] ) ?;
466+ let mut out_batches = collect ( filter) . await ?;
467+ let res = if out_batches. len ( ) == 1 {
468+ out_batches. pop ( ) . unwrap ( )
469+ } else {
470+ RecordBatch :: concat ( & schema, & out_batches) ?
471+ } ;
472+
473+ Ok ( res. columns ( ) . to_vec ( ) )
474+ } else {
475+ Ok ( data)
476+ }
477+ }
478+
300479 fn validate_table_location ( & self ) -> Result < ( ) , CubeError > {
301480 // TODO
302481 // self.query(None)?;
0 commit comments