@@ -51,7 +51,7 @@ use iceberg::inspect::MetadataTableType;
5151use iceberg:: scan:: FileScanTask ;
5252use iceberg:: spec:: TableProperties ;
5353use iceberg:: table:: Table ;
54- use iceberg:: { Catalog , Error , ErrorKind , NamespaceIdent , Result , TableIdent } ;
54+ use iceberg:: { Catalog , Error , ErrorKind , NamespaceIdent , Result , Runtime , TableIdent } ;
5555use metadata_table:: IcebergMetadataTableProvider ;
5656
5757use crate :: error:: to_datafusion_error;
@@ -71,6 +71,10 @@ use crate::physical_plan::write::IcebergWriteExec;
7171///
7272/// For read-only access to a specific snapshot without catalog overhead, use
7373/// [`IcebergStaticTableProvider`] instead.
74+ ///
75+ /// When using a CPU/IO split runtime, pass a [`Runtime`] via
76+ /// [`Self::try_new_with_runtime`] so that catalog IO is dispatched to the IO
77+ /// runtime rather than running on the caller's runtime.
7478#[ derive( Debug , Clone ) ]
7579pub struct IcebergTableProvider {
7680 /// The catalog that manages this table
@@ -79,6 +83,8 @@ pub struct IcebergTableProvider {
7983 table_ident : TableIdent ,
8084 /// A reference-counted arrow `Schema` (cached at construction)
8185 schema : ArrowSchemaRef ,
86+ /// When `Some`, IO in `scan` and `insert_into` is spawned on `runtime.io()`.
87+ runtime : Option < Runtime > ,
8288}
8389
8490impl IcebergTableProvider {
@@ -90,6 +96,17 @@ impl IcebergTableProvider {
9096 catalog : Arc < dyn Catalog > ,
9197 namespace : NamespaceIdent ,
9298 name : impl Into < String > ,
99+ ) -> Result < Self > {
100+ Self :: try_new_with_runtime ( catalog, namespace, name, None ) . await
101+ }
102+
103+ /// Like [`Self::try_new`], but routes catalog IO in `scan` and `insert_into`
104+ /// through `runtime.io()` instead of running inline on the caller's runtime.
105+ pub async fn try_new_with_runtime (
106+ catalog : Arc < dyn Catalog > ,
107+ namespace : NamespaceIdent ,
108+ name : impl Into < String > ,
109+ runtime : Option < Runtime > ,
93110 ) -> Result < Self > {
94111 let table_ident = TableIdent :: new ( namespace, name. into ( ) ) ;
95112
@@ -100,9 +117,25 @@ impl IcebergTableProvider {
100117 catalog,
101118 table_ident,
102119 schema,
120+ runtime,
103121 } )
104122 }
105123
124+ /// Runs `fut` on `self.runtime.io()` if set, otherwise runs it inline.
125+ async fn run_on_io < F , T > ( & self , fut : F ) -> DFResult < T >
126+ where
127+ F : std:: future:: Future < Output = DFResult < T > > + Send + ' static ,
128+ T : Send + ' static ,
129+ {
130+ match & self . runtime {
131+ Some ( rt) => match rt. io ( ) . spawn ( fut) . await {
132+ Ok ( inner) => inner,
133+ Err ( e) => Err ( to_datafusion_error ( e) ) ,
134+ } ,
135+ None => fut. await ,
136+ }
137+ }
138+
106139 pub ( crate ) async fn metadata_table (
107140 & self ,
108141 r#type : MetadataTableType ,
@@ -133,43 +166,58 @@ impl TableProvider for IcebergTableProvider {
133166 filters : & [ Expr ] ,
134167 limit : Option < usize > ,
135168 ) -> DFResult < Arc < dyn ExecutionPlan > > {
136- // Second load: fetch the latest snapshot so scans always reflect current table state.
137- let table = self
138- . catalog
139- . load_table ( & self . table_ident )
140- . await
141- . map_err ( to_datafusion_error) ?;
142-
143- // Build a TableScan mirroring the inputs we'll hand to IcebergTableScan,
144- // so plan_files() uses the same projection/filters the scan will replay in execute().
145- let col_names = projection. map ( |indices| {
146- indices
147- . iter ( )
148- . map ( |& i| self . schema . field ( i) . name ( ) . clone ( ) )
149- . collect :: < Vec < _ > > ( )
150- } ) ;
151-
169+ // Compute the predicate on the caller's runtime (pure CPU, no IO).
152170 let predicate = convert_filters_to_predicate ( filters) ;
153171
154- let mut builder = table. scan ( ) ;
155- builder = match col_names {
156- Some ( names) => builder. select ( names) ,
157- None => builder. select_all ( ) ,
158- } ;
159- if let Some ( pred) = predicate {
160- builder = builder. with_filter ( pred) ;
161- }
162-
163- let tasks: Vec < FileScanTask > = builder
164- . build ( )
165- . map_err ( to_datafusion_error) ?
166- . plan_files ( )
167- . await
168- . map_err ( to_datafusion_error) ?
169- . try_collect :: < Vec < _ > > ( )
170- . await
171- . map_err ( to_datafusion_error) ?;
172+ // Capture everything the IO closure needs; Session is not Send.
173+ let target_partitions = state. config ( ) . target_partitions ( ) ;
174+ let projection_owned: Option < Vec < usize > > = projection. cloned ( ) ;
175+ let catalog = self . catalog . clone ( ) ;
176+ let table_ident = self . table_ident . clone ( ) ;
177+ let arrow_schema = self . schema . clone ( ) ;
178+
179+ // ── IO-bound: reload table + plan files ──────────────────────────────
180+ // Spawned on `self.runtime.io()` when configured, otherwise runs inline.
181+ let ( table, tasks) = self
182+ . run_on_io ( async move {
183+ // Second load: fetch the latest snapshot so scans always reflect
184+ // current table state.
185+ let table = catalog
186+ . load_table ( & table_ident)
187+ . await
188+ . map_err ( to_datafusion_error) ?;
189+
190+ let col_names = projection_owned. as_ref ( ) . map ( |indices| {
191+ indices
192+ . iter ( )
193+ . map ( |& i| arrow_schema. field ( i) . name ( ) . clone ( ) )
194+ . collect :: < Vec < _ > > ( )
195+ } ) ;
196+
197+ let mut builder = table. scan ( ) ;
198+ builder = match col_names {
199+ Some ( names) => builder. select ( names) ,
200+ None => builder. select_all ( ) ,
201+ } ;
202+ if let Some ( pred) = predicate {
203+ builder = builder. with_filter ( pred) ;
204+ }
205+
206+ let tasks: Vec < FileScanTask > = builder
207+ . build ( )
208+ . map_err ( to_datafusion_error) ?
209+ . plan_files ( )
210+ . await
211+ . map_err ( to_datafusion_error) ?
212+ . try_collect :: < Vec < _ > > ( )
213+ . await
214+ . map_err ( to_datafusion_error) ?;
215+
216+ DFResult :: Ok ( ( table, tasks) )
217+ } )
218+ . await ?;
172219
220+ // ── CPU-bound: schema projection + bucketing ──────────────────────────
173221 // Output schema after projection: column indices in `Hash` exprs and any
174222 // Arrow array we hash must reference this schema, not the full table schema.
175223 let output_schema = match projection {
@@ -179,7 +227,6 @@ impl TableProvider for IcebergTableProvider {
179227 } ) ?) ,
180228 } ;
181229
182- let target_partitions = state. config ( ) . target_partitions ( ) ;
183230 // Always produce at least 1 partition so that DataFusion can schedule
184231 // the plan normally and callers can safely call execute(0). An empty
185232 // bucket simply yields an empty record-batch stream.
@@ -232,11 +279,16 @@ impl TableProvider for IcebergTableProvider {
232279 input : Arc < dyn ExecutionPlan > ,
233280 _insert_op : InsertOp ,
234281 ) -> DFResult < Arc < dyn ExecutionPlan > > {
282+ let catalog = self . catalog . clone ( ) ;
283+ let table_ident = self . table_ident . clone ( ) ;
235284 let table = self
236- . catalog
237- . load_table ( & self . table_ident )
238- . await
239- . map_err ( to_datafusion_error) ?;
285+ . run_on_io ( async move {
286+ catalog
287+ . load_table ( & table_ident)
288+ . await
289+ . map_err ( to_datafusion_error)
290+ } )
291+ . await ?;
240292
241293 let partition_spec = table. metadata ( ) . default_partition_spec ( ) ;
242294
0 commit comments