You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by al...@apache.org on 2022/12/14 20:31:11 UTC

[arrow-datafusion] branch master updated: refactor: relax the signature of register_* in SessionContext (#4612)

This is an automated email from the ASF dual-hosted git repository.

alamb pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow-datafusion.git


The following commit(s) were added to refs/heads/master by this push:
     new da0de9d2d refactor: relax the signature of register_* in SessionContext (#4612)
da0de9d2d is described below

commit da0de9d2d60bd6a16c368da78a9a11ba7018a456
Author: Ruihang Xia <wa...@gmail.com>
AuthorDate: Thu Dec 15 04:31:06 2022 +0800

    refactor: relax the signature of register_* in SessionContext (#4612)
    
    Signed-off-by: Ruihang Xia <wa...@gmail.com>
    
    Signed-off-by: Ruihang Xia <wa...@gmail.com>
---
 datafusion-examples/examples/simple_udf.rs            |  2 +-
 datafusion/core/src/execution/context.rs              | 14 +++++++-------
 datafusion/core/src/physical_plan/file_format/json.rs |  2 +-
 datafusion/core/tests/sql/mod.rs                      |  2 +-
 datafusion/core/tests/sql/udf.rs                      |  6 +++---
 datafusion/proto/src/bytes/mod.rs                     |  2 +-
 datafusion/proto/src/lib.rs                           |  4 ++--
 datafusion/proto/src/physical_plan/mod.rs             |  2 +-
 8 files changed, 17 insertions(+), 17 deletions(-)

diff --git a/datafusion-examples/examples/simple_udf.rs b/datafusion-examples/examples/simple_udf.rs
index 7f5fe4a27..c9044a87a 100644
--- a/datafusion-examples/examples/simple_udf.rs
+++ b/datafusion-examples/examples/simple_udf.rs
@@ -59,7 +59,7 @@ fn create_context() -> Result<SessionContext> {
 /// In this example we will declare a single-type, single return type UDF that exponentiates f64, a^b
 #[tokio::main]
 async fn main() -> Result<()> {
-    let mut ctx = create_context()?;
+    let ctx = create_context()?;
 
     // First, declare the actual implementation of the calculation
     let pow = |args: &[ArrayRef]| {
diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs
index 759cc79a8..a73a95401 100644
--- a/datafusion/core/src/execution/context.rs
+++ b/datafusion/core/src/execution/context.rs
@@ -571,7 +571,7 @@ impl SessionContext {
 
     /// Registers a variable provider within this context.
     pub fn register_variable(
-        &mut self,
+        &self,
         variable_type: VarType,
         provider: Arc<dyn VarProvider + Send + Sync>,
     ) {
@@ -588,7 +588,7 @@ impl SessionContext {
     ///
     /// `SELECT MY_FUNC(x)...` will look for a function named `"my_func"`
     /// `SELECT "my_FUNC"(x)` will look for a function named `"my_FUNC"`
-    pub fn register_udf(&mut self, f: ScalarUDF) {
+    pub fn register_udf(&self, f: ScalarUDF) {
         self.state
             .write()
             .scalar_functions
@@ -602,7 +602,7 @@ impl SessionContext {
     ///
     /// `SELECT MY_UDAF(x)...` will look for an aggregate named `"my_udaf"`
     /// `SELECT "my_UDAF"(x)` will look for an aggregate named `"my_UDAF"`
-    pub fn register_udaf(&mut self, f: AggregateUDF) {
+    pub fn register_udaf(&self, f: AggregateUDF) {
         self.state
             .write()
             .aggregate_functions
@@ -638,7 +638,7 @@ impl SessionContext {
 
     /// Creates a [`DataFrame`] for reading an Json data source.
     pub async fn read_json(
-        &mut self,
+        &self,
         table_path: impl AsRef<str>,
         options: NdJsonReadOptions<'_>,
     ) -> Result<Arc<DataFrame>> {
@@ -2085,7 +2085,7 @@ mod tests {
     async fn create_variable_expr() -> Result<()> {
         let tmp_dir = TempDir::new()?;
         let partition_count = 4;
-        let mut ctx = create_ctx(&tmp_dir, partition_count).await?;
+        let ctx = create_ctx(&tmp_dir, partition_count).await?;
 
         let variable_provider = test::variable::SystemVar::new();
         ctx.register_variable(VarType::System, Arc::new(variable_provider));
@@ -2143,7 +2143,7 @@ mod tests {
 
     #[tokio::test]
     async fn case_sensitive_identifiers_user_defined_functions() -> Result<()> {
-        let mut ctx = SessionContext::new();
+        let ctx = SessionContext::new();
         ctx.register_table("t", test::table_with_sequence(1, 1).unwrap())
             .unwrap();
 
@@ -2184,7 +2184,7 @@ mod tests {
 
     #[tokio::test]
     async fn case_sensitive_identifiers_user_defined_aggregates() -> Result<()> {
-        let mut ctx = SessionContext::new();
+        let ctx = SessionContext::new();
         ctx.register_table("t", test::table_with_sequence(1, 1).unwrap())
             .unwrap();
 
diff --git a/datafusion/core/src/physical_plan/file_format/json.rs b/datafusion/core/src/physical_plan/file_format/json.rs
index e507c7d6d..fac394ee3 100644
--- a/datafusion/core/src/physical_plan/file_format/json.rs
+++ b/datafusion/core/src/physical_plan/file_format/json.rs
@@ -303,7 +303,7 @@ mod tests {
         file_compression_type: FileCompressionType,
         store: Arc<dyn ObjectStore>,
     ) {
-        let mut ctx = SessionContext::new();
+        let ctx = SessionContext::new();
         ctx.runtime_env()
             .register_object_store("file", "", store.clone());
         let filename = "1.json";
diff --git a/datafusion/core/tests/sql/mod.rs b/datafusion/core/tests/sql/mod.rs
index 1e1307672..1075041c2 100644
--- a/datafusion/core/tests/sql/mod.rs
+++ b/datafusion/core/tests/sql/mod.rs
@@ -138,7 +138,7 @@ where
 }
 
 fn create_ctx() -> SessionContext {
-    let mut ctx = SessionContext::new();
+    let ctx = SessionContext::new();
 
     // register a custom UDF
     ctx.register_udf(create_udf(
diff --git a/datafusion/core/tests/sql/udf.rs b/datafusion/core/tests/sql/udf.rs
index b71a13cdd..3a150dc7c 100644
--- a/datafusion/core/tests/sql/udf.rs
+++ b/datafusion/core/tests/sql/udf.rs
@@ -53,7 +53,7 @@ async fn scalar_udf() -> Result<()> {
         ],
     )?;
 
-    let mut ctx = SessionContext::new();
+    let ctx = SessionContext::new();
 
     ctx.register_batch("t", batch)?;
 
@@ -138,7 +138,7 @@ async fn simple_udaf() -> Result<()> {
         vec![Arc::new(Int32Array::from_slice([4, 5]))],
     )?;
 
-    let mut ctx = SessionContext::new();
+    let ctx = SessionContext::new();
 
     let provider = MemTable::try_new(Arc::new(schema), vec![vec![batch1], vec![batch2]])?;
     ctx.register_table("t", Arc::new(provider))?;
@@ -205,7 +205,7 @@ fn udaf_as_window_func() -> Result<()> {
         Arc::new(vec![DataType::Int32]),
     );
 
-    let mut context = SessionContext::new();
+    let context = SessionContext::new();
     context.register_table(
         "my_table",
         Arc::new(datafusion::datasource::empty::EmptyTable::new(Arc::new(
diff --git a/datafusion/proto/src/bytes/mod.rs b/datafusion/proto/src/bytes/mod.rs
index 005b76a56..1eb946bd5 100644
--- a/datafusion/proto/src/bytes/mod.rs
+++ b/datafusion/proto/src/bytes/mod.rs
@@ -553,7 +553,7 @@ mod test {
             scalar_fn,
         );
 
-        let mut ctx = SessionContext::new();
+        let ctx = SessionContext::new();
         ctx.register_udf(udf);
 
         ctx
diff --git a/datafusion/proto/src/lib.rs b/datafusion/proto/src/lib.rs
index 5f5975a90..ecafbed3a 100644
--- a/datafusion/proto/src/lib.rs
+++ b/datafusion/proto/src/lib.rs
@@ -1256,7 +1256,7 @@ mod roundtrip_tests {
             filter: Some(Box::new(lit(true))),
         };
 
-        let mut ctx = SessionContext::new();
+        let ctx = SessionContext::new();
         ctx.register_udaf(dummy_agg);
 
         roundtrip_expr_test(test_expr, ctx);
@@ -1281,7 +1281,7 @@ mod roundtrip_tests {
             args: vec![lit("")],
         };
 
-        let mut ctx = SessionContext::new();
+        let ctx = SessionContext::new();
         ctx.register_udf(udf);
 
         roundtrip_expr_test(test_expr, ctx);
diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs
index f890c3ece..f6b6a0bfe 100644
--- a/datafusion/proto/src/physical_plan/mod.rs
+++ b/datafusion/proto/src/physical_plan/mod.rs
@@ -1558,7 +1558,7 @@ mod roundtrip_tests {
         let project =
             ProjectionExec::try_new(vec![(Arc::new(expr), "a".to_string())], input)?;
 
-        let mut ctx = SessionContext::new();
+        let ctx = SessionContext::new();
 
         ctx.register_udf(udf);