@@ -296,7 +296,7 @@ impl PySQLOptions {
296296/// `PySessionContext` is able to plan and execute DataFusion plans. 
297297/// It has a powerful optimizer, a physical planner for local execution, and a 
298298/// multi-threaded execution engine to perform the execution. 
299- #[ pyclass( name = "SessionContext" ,  module = "datafusion" ,  subclass) ]  
299+ #[ pyclass( frozen ,   name = "SessionContext" ,  module = "datafusion" ,  subclass) ]  
300300#[ derive( Clone ) ]  
301301pub  struct  PySessionContext  { 
302302    pub  ctx :  SessionContext , 
@@ -348,7 +348,7 @@ impl PySessionContext {
348348    /// Register an object store with the given name 
349349     #[ pyo3( signature = ( scheme,  store,  host=None ) ) ]  
350350    pub  fn  register_object_store ( 
351-         & mut   self , 
351+         & self , 
352352        scheme :  & str , 
353353        store :  StorageContexts , 
354354        host :  Option < & str > , 
@@ -380,7 +380,7 @@ impl PySessionContext {
380380    schema=None ,  
381381    file_sort_order=None ) ) ]  
382382    pub  fn  register_listing_table ( 
383-         & mut   self , 
383+         & self , 
384384        name :  & str , 
385385        path :  & str , 
386386        table_partition_cols :  Vec < ( String ,  PyArrowType < DataType > ) > , 
@@ -421,22 +421,22 @@ impl PySessionContext {
421421        Ok ( ( ) ) 
422422    } 
423423
424-     pub  fn  register_udtf ( & mut   self ,  func :  PyTableFunction )  { 
424+     pub  fn  register_udtf ( & self ,  func :  PyTableFunction )  { 
425425        let  name = func. name . clone ( ) ; 
426426        let  func = Arc :: new ( func) ; 
427427        self . ctx . register_udtf ( & name,  func) ; 
428428    } 
429429
430430    /// Returns a PyDataFrame whose plan corresponds to the SQL statement. 
431-      pub  fn  sql ( & mut   self ,  query :  & str ,  py :  Python )  -> PyDataFusionResult < PyDataFrame >  { 
431+      pub  fn  sql ( & self ,  query :  & str ,  py :  Python )  -> PyDataFusionResult < PyDataFrame >  { 
432432        let  result = self . ctx . sql ( query) ; 
433433        let  df = wait_for_future ( py,  result) ??; 
434434        Ok ( PyDataFrame :: new ( df) ) 
435435    } 
436436
437437    #[ pyo3( signature = ( query,  options=None ) ) ]  
438438    pub  fn  sql_with_options ( 
439-         & mut   self , 
439+         & self , 
440440        query :  & str , 
441441        options :  Option < PySQLOptions > , 
442442        py :  Python , 
@@ -453,7 +453,7 @@ impl PySessionContext {
453453
454454    #[ pyo3( signature = ( partitions,  name=None ,  schema=None ) ) ]  
455455    pub  fn  create_dataframe ( 
456-         & mut   self , 
456+         & self , 
457457        partitions :  PyArrowType < Vec < Vec < RecordBatch > > > , 
458458        name :  Option < & str > , 
459459        schema :  Option < PyArrowType < Schema > > , 
@@ -488,14 +488,14 @@ impl PySessionContext {
488488    } 
489489
490490    /// Create a DataFrame from an existing logical plan 
491-      pub  fn  create_dataframe_from_logical_plan ( & mut   self ,  plan :  PyLogicalPlan )  -> PyDataFrame  { 
491+      pub  fn  create_dataframe_from_logical_plan ( & self ,  plan :  PyLogicalPlan )  -> PyDataFrame  { 
492492        PyDataFrame :: new ( DataFrame :: new ( self . ctx . state ( ) ,  plan. plan . as_ref ( ) . clone ( ) ) ) 
493493    } 
494494
495495    /// Construct datafusion dataframe from Python list 
496496     #[ pyo3( signature = ( data,  name=None ) ) ]  
497497    pub  fn  from_pylist ( 
498-         & mut   self , 
498+         & self , 
499499        data :  Bound < ' _ ,  PyList > , 
500500        name :  Option < & str > , 
501501    )  -> PyResult < PyDataFrame >  { 
@@ -515,7 +515,7 @@ impl PySessionContext {
515515    /// Construct datafusion dataframe from Python dictionary 
516516     #[ pyo3( signature = ( data,  name=None ) ) ]  
517517    pub  fn  from_pydict ( 
518-         & mut   self , 
518+         & self , 
519519        data :  Bound < ' _ ,  PyDict > , 
520520        name :  Option < & str > , 
521521    )  -> PyResult < PyDataFrame >  { 
@@ -535,7 +535,7 @@ impl PySessionContext {
535535    /// Construct datafusion dataframe from Arrow Table 
536536     #[ pyo3( signature = ( data,  name=None ) ) ]  
537537    pub  fn  from_arrow ( 
538-         & mut   self , 
538+         & self , 
539539        data :  Bound < ' _ ,  PyAny > , 
540540        name :  Option < & str > , 
541541        py :  Python , 
@@ -569,11 +569,7 @@ impl PySessionContext {
569569    /// Construct datafusion dataframe from pandas 
570570     #[ allow( clippy:: wrong_self_convention) ]  
571571    #[ pyo3( signature = ( data,  name=None ) ) ]  
572-     pub  fn  from_pandas ( 
573-         & mut  self , 
574-         data :  Bound < ' _ ,  PyAny > , 
575-         name :  Option < & str > , 
576-     )  -> PyResult < PyDataFrame >  { 
572+     pub  fn  from_pandas ( & self ,  data :  Bound < ' _ ,  PyAny > ,  name :  Option < & str > )  -> PyResult < PyDataFrame >  { 
577573        // Obtain GIL token 
578574        let  py = data. py ( ) ; 
579575
@@ -589,11 +585,7 @@ impl PySessionContext {
589585
590586    /// Construct datafusion dataframe from polars 
591587     #[ pyo3( signature = ( data,  name=None ) ) ]  
592-     pub  fn  from_polars ( 
593-         & mut  self , 
594-         data :  Bound < ' _ ,  PyAny > , 
595-         name :  Option < & str > , 
596-     )  -> PyResult < PyDataFrame >  { 
588+     pub  fn  from_polars ( & self ,  data :  Bound < ' _ ,  PyAny > ,  name :  Option < & str > )  -> PyResult < PyDataFrame >  { 
597589        // Convert Polars dataframe to Arrow Table 
598590        let  table = data. call_method0 ( "to_arrow" ) ?; 
599591
@@ -602,24 +594,20 @@ impl PySessionContext {
602594        Ok ( df) 
603595    } 
604596
605-     pub  fn  register_table ( 
606-         & mut  self , 
607-         name :  & str , 
608-         table :  Bound < ' _ ,  PyAny > , 
609-     )  -> PyDataFusionResult < ( ) >  { 
597+     pub  fn  register_table ( & self ,  name :  & str ,  table :  Bound < ' _ ,  PyAny > )  -> PyDataFusionResult < ( ) >  { 
610598        let  table = PyTable :: new ( & table) ?; 
611599
612600        self . ctx . register_table ( name,  table. table ) ?; 
613601        Ok ( ( ) ) 
614602    } 
615603
616-     pub  fn  deregister_table ( & mut   self ,  name :  & str )  -> PyDataFusionResult < ( ) >  { 
604+     pub  fn  deregister_table ( & self ,  name :  & str )  -> PyDataFusionResult < ( ) >  { 
617605        self . ctx . deregister_table ( name) ?; 
618606        Ok ( ( ) ) 
619607    } 
620608
621609    pub  fn  register_catalog_provider ( 
622-         & mut   self , 
610+         & self , 
623611        name :  & str , 
624612        provider :  Bound < ' _ ,  PyAny > , 
625613    )  -> PyDataFusionResult < ( ) >  { 
@@ -648,7 +636,7 @@ impl PySessionContext {
648636
649637    /// Construct datafusion dataframe from Arrow Table 
650638     pub  fn  register_table_provider ( 
651-         & mut   self , 
639+         & self , 
652640        name :  & str , 
653641        provider :  Bound < ' _ ,  PyAny > , 
654642    )  -> PyDataFusionResult < ( ) >  { 
@@ -657,7 +645,7 @@ impl PySessionContext {
657645    } 
658646
659647    pub  fn  register_record_batches ( 
660-         & mut   self , 
648+         & self , 
661649        name :  & str , 
662650        partitions :  PyArrowType < Vec < Vec < RecordBatch > > > , 
663651    )  -> PyDataFusionResult < ( ) >  { 
@@ -675,7 +663,7 @@ impl PySessionContext {
675663                        schema=None ,  
676664                        file_sort_order=None ) ) ]  
677665    pub  fn  register_parquet ( 
678-         & mut   self , 
666+         & self , 
679667        name :  & str , 
680668        path :  & str , 
681669        table_partition_cols :  Vec < ( String ,  PyArrowType < DataType > ) > , 
@@ -718,7 +706,7 @@ impl PySessionContext {
718706                        file_extension=".csv" ,  
719707                        file_compression_type=None ) ) ]  
720708    pub  fn  register_csv ( 
721-         & mut   self , 
709+         & self , 
722710        name :  & str , 
723711        path :  & Bound < ' _ ,  PyAny > , 
724712        schema :  Option < PyArrowType < Schema > > , 
@@ -766,7 +754,7 @@ impl PySessionContext {
766754                        table_partition_cols=vec![ ] ,  
767755                        file_compression_type=None ) ) ]  
768756    pub  fn  register_json ( 
769-         & mut   self , 
757+         & self , 
770758        name :  & str , 
771759        path :  PathBuf , 
772760        schema :  Option < PyArrowType < Schema > > , 
@@ -805,7 +793,7 @@ impl PySessionContext {
805793                        file_extension=".avro" ,  
806794                        table_partition_cols=vec![ ] ) ) ]  
807795    pub  fn  register_avro ( 
808-         & mut   self , 
796+         & self , 
809797        name :  & str , 
810798        path :  PathBuf , 
811799        schema :  Option < PyArrowType < Schema > > , 
@@ -846,17 +834,17 @@ impl PySessionContext {
846834        Ok ( ( ) ) 
847835    } 
848836
849-     pub  fn  register_udf ( & mut   self ,  udf :  PyScalarUDF )  -> PyResult < ( ) >  { 
837+     pub  fn  register_udf ( & self ,  udf :  PyScalarUDF )  -> PyResult < ( ) >  { 
850838        self . ctx . register_udf ( udf. function ) ; 
851839        Ok ( ( ) ) 
852840    } 
853841
854-     pub  fn  register_udaf ( & mut   self ,  udaf :  PyAggregateUDF )  -> PyResult < ( ) >  { 
842+     pub  fn  register_udaf ( & self ,  udaf :  PyAggregateUDF )  -> PyResult < ( ) >  { 
855843        self . ctx . register_udaf ( udaf. function ) ; 
856844        Ok ( ( ) ) 
857845    } 
858846
859-     pub  fn  register_udwf ( & mut   self ,  udwf :  PyWindowUDF )  -> PyResult < ( ) >  { 
847+     pub  fn  register_udwf ( & self ,  udwf :  PyWindowUDF )  -> PyResult < ( ) >  { 
860848        self . ctx . register_udwf ( udwf. function ) ; 
861849        Ok ( ( ) ) 
862850    } 
@@ -928,7 +916,7 @@ impl PySessionContext {
928916    #[ allow( clippy:: too_many_arguments) ]  
929917    #[ pyo3( signature = ( path,  schema=None ,  schema_infer_max_records=1000 ,  file_extension=".json" ,  table_partition_cols=vec![ ] ,  file_compression_type=None ) ) ]  
930918    pub  fn  read_json ( 
931-         & mut   self , 
919+         & self , 
932920        path :  PathBuf , 
933921        schema :  Option < PyArrowType < Schema > > , 
934922        schema_infer_max_records :  usize , 
0 commit comments