@@ -18,7 +18,7 @@ use arrow::pyarrow::PyArrowType;
1818use chrono:: { DateTime , Duration , FixedOffset , Utc } ;
1919use datafusion_ffi:: table_provider:: FFI_TableProvider ;
2020use delta_kernel:: expressions:: Scalar ;
21- use delta_kernel:: schema:: StructField ;
21+ use delta_kernel:: schema:: { MetadataValue , StructField } ;
2222use deltalake:: arrow:: compute:: concat_batches;
2323use deltalake:: arrow:: ffi_stream:: { ArrowArrayStreamReader , FFI_ArrowArrayStream } ;
2424use deltalake:: arrow:: record_batch:: { RecordBatch , RecordBatchIterator } ;
@@ -63,13 +63,6 @@ use error::DeltaError;
6363use futures:: future:: join_all;
6464use tracing:: log:: * ;
6565
66- use pyo3:: exceptions:: { PyRuntimeError , PyValueError } ;
67- use pyo3:: prelude:: * ;
68- use pyo3:: pybacked:: PyBackedStr ;
69- use pyo3:: types:: { PyCapsule , PyDict , PyFrozenSet } ;
70- use serde_json:: { Map , Value } ;
71- use uuid:: Uuid ;
72-
7366use crate :: error:: DeltaProtocolError ;
7467use crate :: error:: PythonError ;
7568use crate :: features:: TableFeatures ;
@@ -78,6 +71,14 @@ use crate::merge::PyMergeBuilder;
7871use crate :: query:: PyQueryBuilder ;
7972use crate :: schema:: { schema_to_pyobject, Field } ;
8073use crate :: utils:: rt;
74+ use deltalake:: operations:: update_field_metadata:: UpdateFieldMetadataBuilder ;
75+ use deltalake:: protocol:: DeltaOperation :: UpdateFieldMetadata ;
76+ use pyo3:: exceptions:: { PyRuntimeError , PyValueError } ;
77+ use pyo3:: prelude:: * ;
78+ use pyo3:: pybacked:: PyBackedStr ;
79+ use pyo3:: types:: { PyCapsule , PyDict , PyFrozenSet } ;
80+ use serde_json:: { Map , Value } ;
81+ use uuid:: Uuid ;
8182
8283#[ cfg( all( target_family = "unix" , not( target_os = "emscripten" ) ) ) ]
8384use jemallocator:: Jemalloc ;
@@ -1521,6 +1522,43 @@ impl RawDeltaTable {
15211522 }
15221523 }
15231524
1525+ #[ pyo3( signature = ( field_name, metadata, commit_properties=None , post_commithook_properties=None ) ) ]
1526+ pub fn set_column_metadata (
1527+ & self ,
1528+ py : Python ,
1529+ field_name : & str ,
1530+ metadata : HashMap < String , String > ,
1531+ commit_properties : Option < PyCommitProperties > ,
1532+ post_commithook_properties : Option < PyPostCommitHookProperties > ,
1533+ ) -> PyResult < ( ) > {
1534+ let table = py. allow_threads ( || {
1535+ let mut cmd = UpdateFieldMetadataBuilder :: new ( self . log_store ( ) ?, self . cloned_state ( ) ?) ;
1536+
1537+ cmd = cmd. with_field_name ( field_name) . with_metadata (
1538+ metadata
1539+ . iter ( )
1540+ . map ( |( k, v) | ( k. clone ( ) , MetadataValue :: String ( v. clone ( ) ) ) )
1541+ . collect ( ) ,
1542+ ) ;
1543+
1544+ if let Some ( commit_properties) =
1545+ maybe_create_commit_properties ( commit_properties, post_commithook_properties)
1546+ {
1547+ cmd = cmd. with_commit_properties ( commit_properties)
1548+ }
1549+
1550+ if self . log_store ( ) ?. name ( ) == "LakeFSLogStore" {
1551+ cmd = cmd. with_custom_execute_handler ( Arc :: new ( LakeFSCustomExecuteHandler { } ) )
1552+ }
1553+
1554+ rt ( ) . block_on ( cmd. into_future ( ) )
1555+ . map_err ( PythonError :: from)
1556+ . map_err ( PyErr :: from)
1557+ } ) ?;
1558+ self . set_state ( table. state ) ?;
1559+ Ok ( ( ) )
1560+ }
1561+
15241562 fn __datafusion_table_provider__ < ' py > (
15251563 & self ,
15261564 py : Python < ' py > ,
0 commit comments