@@ -24,17 +24,16 @@ use std::sync::Arc;
24
24
use async_trait:: async_trait;
25
25
use datafusion:: arrow:: datatypes:: SchemaRef as ArrowSchemaRef ;
26
26
use datafusion:: catalog:: Session ;
27
- use datafusion:: common:: DataFusionError ;
28
27
use datafusion:: datasource:: { TableProvider , TableType } ;
29
- use datafusion:: error:: Result as DFResult ;
28
+ use datafusion:: error:: { DataFusionError , Result as DFResult } ;
30
29
use datafusion:: logical_expr:: dml:: InsertOp ;
31
30
use datafusion:: logical_expr:: { Expr , TableProviderFilterPushDown } ;
32
31
use datafusion:: physical_plan:: ExecutionPlan ;
33
32
use datafusion:: physical_plan:: coalesce_partitions:: CoalescePartitionsExec ;
34
33
use iceberg:: arrow:: schema_to_arrow_schema;
35
34
use iceberg:: inspect:: MetadataTableType ;
36
35
use iceberg:: table:: Table ;
37
- use iceberg:: { Catalog , Error , ErrorKind , NamespaceIdent , Result , TableIdent } ;
36
+ use iceberg:: { Catalog , Error , ErrorKind , Result , TableIdent } ;
38
37
use metadata_table:: IcebergMetadataTableProvider ;
39
38
40
39
use crate :: physical_plan:: commit:: IcebergCommitExec ;
@@ -43,7 +42,7 @@ use crate::physical_plan::write::IcebergWriteExec;
43
42
44
43
/// Represents a [`TableProvider`] for the Iceberg [`Catalog`],
45
44
/// managing access to a [`Table`].
46
- #[ derive( Debug , Clone ) ]
45
+ #[ derive( Clone ) ]
47
46
pub struct IcebergTableProvider {
48
47
/// A table in the catalog.
49
48
table : Table ,
@@ -55,6 +54,16 @@ pub struct IcebergTableProvider {
55
54
catalog : Option < Arc < dyn Catalog > > ,
56
55
}
57
56
57
+ impl std:: fmt:: Debug for IcebergTableProvider {
58
+ fn fmt ( & self , f : & mut std:: fmt:: Formatter < ' _ > ) -> std:: fmt:: Result {
59
+ f. debug_struct ( "IcebergTableProvider" )
60
+ . field ( "table" , & self . table )
61
+ . field ( "snapshot_id" , & self . snapshot_id )
62
+ . field ( "schema" , & self . schema )
63
+ . finish_non_exhaustive ( )
64
+ }
65
+ }
66
+
58
67
impl IcebergTableProvider {
59
68
pub ( crate ) fn new ( table : Table , schema : ArrowSchemaRef ) -> Self {
60
69
IcebergTableProvider {
@@ -67,13 +76,8 @@ impl IcebergTableProvider {
67
76
/// Asynchronously tries to construct a new [`IcebergTableProvider`]
68
77
/// using the given client and table name to fetch an actual [`Table`]
69
78
/// in the provided namespace.
70
- pub ( crate ) async fn try_new (
71
- client : Arc < dyn Catalog > ,
72
- namespace : NamespaceIdent ,
73
- name : impl Into < String > ,
74
- ) -> Result < Self > {
75
- let ident = TableIdent :: new ( namespace, name. into ( ) ) ;
76
- let table = client. load_table ( & ident) . await ?;
79
+ pub async fn try_new ( client : Arc < dyn Catalog > , table_name : TableIdent ) -> Result < Self > {
80
+ let table = client. load_table ( & table_name) . await ?;
77
81
78
82
let schema = Arc :: new ( schema_to_arrow_schema ( table. metadata ( ) . current_schema ( ) ) ?) ;
79
83
@@ -151,8 +155,19 @@ impl TableProvider for IcebergTableProvider {
151
155
filters : & [ Expr ] ,
152
156
_limit : Option < usize > ,
153
157
) -> DFResult < Arc < dyn ExecutionPlan > > {
158
+ // Get the latest table metadata from the catalog if it exists
159
+ let table = if let Some ( catalog) = & self . catalog {
160
+ catalog
161
+ . load_table ( self . table . identifier ( ) )
162
+ . await
163
+ . map_err ( |e| {
164
+ DataFusionError :: Execution ( format ! ( "Error getting Iceberg table metadata: {e}" ) )
165
+ } ) ?
166
+ } else {
167
+ self . table . clone ( )
168
+ } ;
154
169
Ok ( Arc :: new ( IcebergTableScan :: new (
155
- self . table . clone ( ) ,
170
+ table,
156
171
self . snapshot_id ,
157
172
self . schema . clone ( ) ,
158
173
projection,
0 commit comments