@@ -20,35 +20,29 @@ use std::sync::Arc;
20
20
21
21
use arrow:: array:: { RecordBatch , StringArray } ;
22
22
use arrow:: datatypes:: { DataType , Field , FieldRef , Schema , SchemaRef } ;
23
- use async_trait:: async_trait;
24
23
25
24
use datafusion:: assert_batches_eq;
26
- use datafusion:: catalog:: memory:: DataSourceExec ;
27
- use datafusion:: catalog:: { Session , TableProvider } ;
28
25
use datafusion:: common:: tree_node:: {
29
26
Transformed , TransformedResult , TreeNode , TreeNodeRecursion ,
30
27
} ;
31
- use datafusion:: common:: { assert_contains, DFSchema , Result } ;
32
- use datafusion:: datasource:: listing:: PartitionedFile ;
33
- use datafusion:: datasource:: physical_plan:: { FileScanConfigBuilder , ParquetSource } ;
28
+ use datafusion:: common:: { assert_contains, Result } ;
29
+ use datafusion:: datasource:: listing:: {
30
+ ListingTable , ListingTableConfig , ListingTableUrl ,
31
+ } ;
34
32
use datafusion:: execution:: context:: SessionContext ;
35
33
use datafusion:: execution:: object_store:: ObjectStoreUrl ;
36
- use datafusion:: logical_expr:: utils:: conjunction;
37
34
use datafusion:: logical_expr:: {
38
- ColumnarValue , Expr , ScalarFunctionArgs , ScalarUDF , ScalarUDFImpl , Signature ,
39
- TableProviderFilterPushDown , TableType , Volatility ,
35
+ ColumnarValue , ScalarFunctionArgs , ScalarUDF , ScalarUDFImpl , Signature , Volatility ,
40
36
} ;
41
37
use datafusion:: parquet:: arrow:: ArrowWriter ;
42
38
use datafusion:: parquet:: file:: properties:: WriterProperties ;
43
39
use datafusion:: physical_expr:: PhysicalExpr ;
44
40
use datafusion:: physical_expr:: { expressions, ScalarFunctionExpr } ;
45
- use datafusion:: physical_plan:: ExecutionPlan ;
46
- use datafusion:: prelude:: { lit, SessionConfig } ;
41
+ use datafusion:: prelude:: SessionConfig ;
47
42
use datafusion:: scalar:: ScalarValue ;
48
43
use datafusion_physical_expr_adapter:: {
49
44
DefaultPhysicalExprAdapterFactory , PhysicalExprAdapter , PhysicalExprAdapterFactory ,
50
45
} ;
51
- use futures:: StreamExt ;
52
46
use object_store:: memory:: InMemory ;
53
47
use object_store:: path:: Path ;
54
48
use object_store:: { ObjectStore , PutPayload } ;
@@ -95,23 +89,29 @@ async fn main() -> Result<()> {
95
89
let payload = PutPayload :: from_bytes ( buf. into ( ) ) ;
96
90
store. put ( & path, payload) . await ?;
97
91
98
- // Create a custom table provider that rewrites struct field access
99
- let table_provider = Arc :: new ( ExampleTableProvider :: new ( table_schema) ) ;
100
-
101
92
// Set up query execution
102
93
let mut cfg = SessionConfig :: new ( ) ;
103
94
cfg. options_mut ( ) . execution . parquet . pushdown_filters = true ;
104
95
let ctx = SessionContext :: new_with_config ( cfg) ;
105
-
106
- // Register our table
107
- ctx. register_table ( "structs" , table_provider) ?;
108
- ctx. register_udf ( ScalarUDF :: new_from_impl ( JsonGetStr :: default ( ) ) ) ;
109
-
110
96
ctx. runtime_env ( ) . register_object_store (
111
97
ObjectStoreUrl :: parse ( "memory://" ) ?. as_ref ( ) ,
112
98
Arc :: new ( store) ,
113
99
) ;
114
100
101
+ // Create a custom table provider that rewrites struct field access
102
+ let listing_table_config =
103
+ ListingTableConfig :: new ( ListingTableUrl :: parse ( "memory:///example.parquet" ) ?)
104
+ . infer_options ( & ctx. state ( ) )
105
+ . await ?
106
+ . with_schema ( table_schema)
107
+ . with_expr_adapter_factory ( Arc :: new ( ShreddedJsonRewriterFactory ) ) ;
108
+ let table = ListingTable :: try_new ( listing_table_config) . unwrap ( ) ;
109
+ let table_provider = Arc :: new ( table) ;
110
+
111
+ // Register our table
112
+ ctx. register_table ( "structs" , table_provider) ?;
113
+ ctx. register_udf ( ScalarUDF :: new_from_impl ( JsonGetStr :: default ( ) ) ) ;
114
+
115
115
println ! ( "\n === Showing all data ===" ) ;
116
116
let batches = ctx. sql ( "SELECT * FROM structs" ) . await ?. collect ( ) . await ?;
117
117
arrow:: util:: pretty:: print_batches ( & batches) ?;
@@ -191,96 +191,6 @@ fn create_sample_record_batch(file_schema: &Schema) -> RecordBatch {
191
191
. unwrap ( )
192
192
}
193
193
194
- /// Custom TableProvider that uses a StructFieldRewriter
195
- #[ derive( Debug ) ]
196
- struct ExampleTableProvider {
197
- schema : SchemaRef ,
198
- }
199
-
200
- impl ExampleTableProvider {
201
- fn new ( schema : SchemaRef ) -> Self {
202
- Self { schema }
203
- }
204
- }
205
-
206
- #[ async_trait]
207
- impl TableProvider for ExampleTableProvider {
208
- fn as_any ( & self ) -> & dyn Any {
209
- self
210
- }
211
-
212
- fn schema ( & self ) -> SchemaRef {
213
- self . schema . clone ( )
214
- }
215
-
216
- fn table_type ( & self ) -> TableType {
217
- TableType :: Base
218
- }
219
-
220
- fn supports_filters_pushdown (
221
- & self ,
222
- filters : & [ & Expr ] ,
223
- ) -> Result < Vec < TableProviderFilterPushDown > > {
224
- // Implementers can choose to mark these filters as exact or inexact.
225
- // If marked as exact they cannot have false positives and must always be applied.
226
- // If marked as Inexact they can have false positives and at runtime the rewriter
227
- // can decide to not rewrite / ignore some filters since they will be re-evaluated upstream.
228
- // For the purposes of this example we mark them as Exact to demonstrate the rewriter is working and the filtering is not being re-evaluated upstream.
229
- Ok ( vec ! [ TableProviderFilterPushDown :: Exact ; filters. len( ) ] )
230
- }
231
-
232
- async fn scan (
233
- & self ,
234
- state : & dyn Session ,
235
- projection : Option < & Vec < usize > > ,
236
- filters : & [ Expr ] ,
237
- limit : Option < usize > ,
238
- ) -> Result < Arc < dyn ExecutionPlan > > {
239
- let schema = self . schema . clone ( ) ;
240
- let df_schema = DFSchema :: try_from ( schema. clone ( ) ) ?;
241
- let filter = state. create_physical_expr (
242
- conjunction ( filters. iter ( ) . cloned ( ) ) . unwrap_or_else ( || lit ( true ) ) ,
243
- & df_schema,
244
- ) ?;
245
-
246
- let parquet_source = ParquetSource :: default ( )
247
- . with_predicate ( filter)
248
- . with_pushdown_filters ( true ) ;
249
-
250
- let object_store_url = ObjectStoreUrl :: parse ( "memory://" ) ?;
251
-
252
- let store = state. runtime_env ( ) . object_store ( object_store_url) ?;
253
-
254
- let mut files = vec ! [ ] ;
255
- let mut listing = store. list ( None ) ;
256
- while let Some ( file) = listing. next ( ) . await {
257
- if let Ok ( file) = file {
258
- files. push ( file) ;
259
- }
260
- }
261
-
262
- let file_group = files
263
- . iter ( )
264
- . map ( |file| PartitionedFile :: new ( file. location . clone ( ) , file. size ) )
265
- . collect ( ) ;
266
-
267
- let file_scan_config = FileScanConfigBuilder :: new (
268
- ObjectStoreUrl :: parse ( "memory://" ) ?,
269
- schema,
270
- Arc :: new ( parquet_source) ,
271
- )
272
- . with_projection ( projection. cloned ( ) )
273
- . with_limit ( limit)
274
- . with_file_group ( file_group)
275
- // if the rewriter needs a reference to the table schema you can bind self.schema() here
276
- . with_expr_adapter ( Some ( Arc :: new ( ShreddedJsonRewriterFactory ) as _ ) ) ;
277
-
278
- Ok ( Arc :: new ( DataSourceExec :: new ( Arc :: new (
279
- file_scan_config. build ( ) ,
280
- ) ) ) )
281
- }
282
- }
283
-
284
194
/// Scalar UDF that uses serde_json to access json fields
285
195
#[ derive( Debug , PartialEq , Eq , Hash ) ]
286
196
pub struct JsonGetStr {
0 commit comments