You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "thinkharderdev (via GitHub)" <gi...@apache.org> on 2023/02/07 21:55:23 UTC

[GitHub] [arrow-ballista] thinkharderdev opened a new pull request, #658: Cluster state refactor Part 2

thinkharderdev opened a new pull request, #658:
URL: https://github.com/apache/arrow-ballista/pull/658

   # Which issue does this PR close?
   
   <!--
   We generally require a GitHub issue to be filed for all bug fixes and enhancements and this helps us generate change logs for our releases. You can link an issue to this PR using the GitHub syntax. For example `Closes #123` indicates that this PR will close issue #123.
   -->
   
   Closes #554 
   
    # Rationale for this change
   <!--
    Why are you proposing this change? If this is already explained clearly in the issue then this section is not needed.
    Explaining clearly why changes are proposed helps reviewers understand your changes and offer better suggestions for fixes.  
   -->
   
   See original ticket for full description but the tl;dr; is:
   
   1. The existing interface for managing shared state among schedulers is too low-level and both pushes too much complexity into other layers and prevents us from taking advanatage of data store specific freatures (atomics, transactions, etc) to avoid locking in the application layer. 
   2. By forcing everything through the KV interface we force all the serialization overhead even if we are only using in-memory state. 
   
   # What changes are included in this PR?
   <!--
   There is no need to duplicate the description in the issue here but it is sometimes worth providing a summary of the individual changes in this PR.
   -->
   
   There are a lot of changes because the `StateBackendClient` was baked in everywhere but at a high level:
   
   1. Create `JobState` trait to complement `ClusterState` created in last PR. This in the interface for storing/managing global state w/r/t jobs and sessions.
   2. Refactor in-memory implementation to not serialize things to protobuf
   3. Provide `KeyValueState` implementation based on existing KV interface so we can continue to use etcd and sled for state without any changes.
   4. Take advantage of curated job architecture to minimize distributed locking. Since jobs are owned by a single scheduler we don't need to lock beyond the node level. 
   5. Cleanup configs.
   6. Move everything related to state the `ballista_scheduler::cluster` module
   
   I removed the separate configs for `cluster-backend` and `config-backend` because it seems confusing. So currently if you want to use different implementations for the `ClusterState` and `JobState` you would need to write your own entrypoint. We might introduce standard "profiles" (standalone/ha/etc) for setting up the two state backends on different implementations but for now it seems too noisy config wise.
   
   NOTE: I was not able to run the integration tests using the standard configuration due to weird timeouts. When using push scheduling everything worked fine so not sure what's going on. I also checked on `main` and see the same issue so maybe something related to the M1 macbook?
   
   # Are there any user-facing changes?
   <!--
   If there are user-facing changes then we may require documentation to be updated before approving the PR.
   -->
   
   The storage layout for etcd and sled are simplified so existing state would not be valid after upgrade. 
   
   <!--
   If there are any breaking changes to public APIs, please add the `api change` label.
   -->
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-ballista] andygrove commented on a diff in pull request #658: Cluster state refactor Part 2

Posted by "andygrove (via GitHub)" <gi...@apache.org>.
andygrove commented on code in PR #658:
URL: https://github.com/apache/arrow-ballista/pull/658#discussion_r1102847694


##########
ballista/scheduler/src/bin/main.rs:
##########
@@ -117,10 +100,13 @@ async fn main() -> Result<()> {
             .init();
     }
 
-    let addr = format!("{bind_host}:{port}");
+    let addr = format!("{}:{}", opt.bind_host, opt.bind_port);
     let addr = addr.parse()?;
 
     let config = SchedulerConfig {
+        namespace: opt.namespace,
+        external_host: opt.external_host,

Review Comment:
   It looks like this PR drops `bind_host`? That can be different from `external_host`, so I'm not sure we can do that.
   
   Or maybe that is handled somewhere else in this PR and I haven't seen it yet?
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-ballista] thinkharderdev commented on pull request #658: Cluster state refactor Part 2

Posted by "thinkharderdev (via GitHub)" <gi...@apache.org>.
thinkharderdev commented on PR #658:
URL: https://github.com/apache/arrow-ballista/pull/658#issuecomment-1421511250

   @mingmwang @yahoNanJing @Dandandan 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-ballista] thinkharderdev commented on a diff in pull request #658: Cluster state refactor Part 2

Posted by "thinkharderdev (via GitHub)" <gi...@apache.org>.
thinkharderdev commented on code in PR #658:
URL: https://github.com/apache/arrow-ballista/pull/658#discussion_r1166685343


##########
ballista/scheduler/src/state/task_manager.rs:
##########
@@ -738,17 +639,18 @@ impl<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> TaskManager<T, U>
     }
 
     /// Get the `ExecutionGraph` for the given job ID from cache
-    pub(crate) async fn get_active_execution_graph(
+    pub(crate) fn get_active_execution_graph(
         &self,
         job_id: &str,
     ) -> Option<Arc<RwLock<ExecutionGraph>>> {
         self.active_job_cache
             .get(job_id)
-            .map(|value| value.execution_graph.clone())
+            .as_deref()
+            .map(|cached| cached.execution_graph.clone())
     }
 
     /// Remove the `ExecutionGraph` for the given job ID from cache
-    pub(crate) async fn remove_active_execution_graph(

Review Comment:
   Sorry, by "global state" I mean state shared across multiple schedulers. The `active_job_cache` is local to a single scheduler and and a purely in-memory data structure



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-ballista] thinkharderdev commented on a diff in pull request #658: Cluster state refactor Part 2

Posted by "thinkharderdev (via GitHub)" <gi...@apache.org>.
thinkharderdev commented on code in PR #658:
URL: https://github.com/apache/arrow-ballista/pull/658#discussion_r1099970688


##########
ballista/scheduler/src/cluster/mod.rs:
##########
@@ -0,0 +1,443 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+pub mod event;
+pub mod kv;
+pub mod memory;
+pub mod storage;
+
+#[cfg(test)]
+#[allow(clippy::uninlined_format_args)]

Review Comment:
   Not sure what else to do here. Clippy complains when I inline variables in the format string in the `assert` and also complains when they are not inlined so just disabling this lint for the module. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-ballista] Ted-Jiang commented on a diff in pull request #658: Cluster state refactor Part 2

Posted by "Ted-Jiang (via GitHub)" <gi...@apache.org>.
Ted-Jiang commented on code in PR #658:
URL: https://github.com/apache/arrow-ballista/pull/658#discussion_r1166481587


##########
ballista/scheduler/src/state/task_manager.rs:
##########
@@ -738,17 +639,18 @@ impl<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> TaskManager<T, U>
     }
 
     /// Get the `ExecutionGraph` for the given job ID from cache
-    pub(crate) async fn get_active_execution_graph(
+    pub(crate) fn get_active_execution_graph(
         &self,
         job_id: &str,
     ) -> Option<Arc<RwLock<ExecutionGraph>>> {
         self.active_job_cache
             .get(job_id)
-            .map(|value| value.execution_graph.clone())
+            .as_deref()
+            .map(|cached| cached.execution_graph.clone())
     }
 
     /// Remove the `ExecutionGraph` for the given job ID from cache
-    pub(crate) async fn remove_active_execution_graph(

Review Comment:
   I think `active_job_cache ` here is a global states here in `TaskManager` ->  `SchedulerState`, Is there something wrong ?  plz tell me 🤔 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-ballista] thinkharderdev commented on a diff in pull request #658: Cluster state refactor Part 2

Posted by "thinkharderdev (via GitHub)" <gi...@apache.org>.
thinkharderdev commented on code in PR #658:
URL: https://github.com/apache/arrow-ballista/pull/658#discussion_r1165312109


##########
ballista/scheduler/src/state/task_manager.rs:
##########
@@ -738,17 +639,18 @@ impl<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> TaskManager<T, U>
     }
 
     /// Get the `ExecutionGraph` for the given job ID from cache
-    pub(crate) async fn get_active_execution_graph(
+    pub(crate) fn get_active_execution_graph(
         &self,
         job_id: &str,
     ) -> Option<Arc<RwLock<ExecutionGraph>>> {
         self.active_job_cache
             .get(job_id)
-            .map(|value| value.execution_graph.clone())
+            .as_deref()
+            .map(|cached| cached.execution_graph.clone())
     }
 
     /// Remove the `ExecutionGraph` for the given job ID from cache
-    pub(crate) async fn remove_active_execution_graph(

Review Comment:
   This function is just removing the `ExecutionGraph` from the local active cache so it doesn't need to be async. We only need an async function when we modify the global state. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-ballista] thinkharderdev commented on pull request #658: Cluster state refactor Part 2

Posted by "thinkharderdev (via GitHub)" <gi...@apache.org>.
thinkharderdev commented on PR #658:
URL: https://github.com/apache/arrow-ballista/pull/658#issuecomment-1426056366

   I'll plan on merging this weekend unless someone else would like more time to review. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-ballista] andygrove commented on a diff in pull request #658: Cluster state refactor Part 2

Posted by "andygrove (via GitHub)" <gi...@apache.org>.
andygrove commented on code in PR #658:
URL: https://github.com/apache/arrow-ballista/pull/658#discussion_r1102848846


##########
ballista/scheduler/src/cluster/storage/sled.rs:
##########
@@ -305,33 +305,33 @@ mod tests {
         Ok(())
     }
 
-    #[tokio::test]
-    async fn multiple_operation() -> Result<(), Box<dyn std::error::Error>> {
-        let client = create_instance()?;
-        let key = "key".to_string();
-        let value = "value".as_bytes().to_vec();
-        let locks = client
-            .acquire_locks(vec![(Keyspace::ActiveJobs, ""), (Keyspace::Slots, "")])
-            .await?;
-
-        let _r: ballista_core::error::Result<()> = with_locks(locks, async {
-            let txn_ops = vec![
-                (Operation::Put(value.clone()), Keyspace::Slots, key.clone()),
-                (
-                    Operation::Put(value.clone()),
-                    Keyspace::ActiveJobs,
-                    key.clone(),
-                ),
-            ];
-            client.apply_txn(txn_ops).await?;
-            Ok(())
-        })
-        .await;
-
-        assert_eq!(client.get(Keyspace::Slots, key.as_str()).await?, value);
-        assert_eq!(client.get(Keyspace::ActiveJobs, key.as_str()).await?, value);
-        Ok(())
-    }
+    // #[tokio::test]

Review Comment:
   Is this test intentionally commented out? Should we add a link to an issue explaining why?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-ballista] andygrove commented on a diff in pull request #658: Cluster state refactor Part 2

Posted by "andygrove (via GitHub)" <gi...@apache.org>.
andygrove commented on code in PR #658:
URL: https://github.com/apache/arrow-ballista/pull/658#discussion_r1102852483


##########
ballista/scheduler/src/bin/main.rs:
##########
@@ -117,10 +100,13 @@ async fn main() -> Result<()> {
             .init();
     }
 
-    let addr = format!("{bind_host}:{port}");
+    let addr = format!("{}:{}", opt.bind_host, opt.bind_port);
     let addr = addr.parse()?;
 
     let config = SchedulerConfig {
+        namespace: opt.namespace,
+        external_host: opt.external_host,

Review Comment:
   It looks like `bind_host` is not currently used in the scheduler, but it is in the executor, so this seems like an oversight.
   
   cc @avantgardnerio who may know have opinions here



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-ballista] avantgardnerio commented on a diff in pull request #658: Cluster state refactor Part 2

Posted by "avantgardnerio (via GitHub)" <gi...@apache.org>.
avantgardnerio commented on code in PR #658:
URL: https://github.com/apache/arrow-ballista/pull/658#discussion_r1102921872


##########
ballista/scheduler/src/bin/main.rs:
##########
@@ -117,10 +100,13 @@ async fn main() -> Result<()> {
             .init();
     }
 
-    let addr = format!("{bind_host}:{port}");
+    let addr = format!("{}:{}", opt.bind_host, opt.bind_port);
     let addr = addr.parse()?;
 
     let config = SchedulerConfig {
+        namespace: opt.namespace,
+        external_host: opt.external_host,

Review Comment:
   I stand corrected - it appears to have morphed into its own `advertise_flight_sql_endpoint`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-ballista] thinkharderdev commented on pull request #658: Cluster state refactor Part 2

Posted by "thinkharderdev (via GitHub)" <gi...@apache.org>.
thinkharderdev commented on PR #658:
URL: https://github.com/apache/arrow-ballista/pull/658#issuecomment-1422653896

   Not sure why this is failing (https://github.com/apache/arrow-ballista/actions/runs/4123347318/jobs/7121701363) but it doesn't seem to be an issue with the actual formatting


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-ballista] thinkharderdev commented on a diff in pull request #658: Cluster state refactor Part 2

Posted by "thinkharderdev (via GitHub)" <gi...@apache.org>.
thinkharderdev commented on code in PR #658:
URL: https://github.com/apache/arrow-ballista/pull/658#discussion_r1102894941


##########
ballista/scheduler/src/bin/main.rs:
##########
@@ -117,10 +100,13 @@ async fn main() -> Result<()> {
             .init();
     }
 
-    let addr = format!("{bind_host}:{port}");
+    let addr = format!("{}:{}", opt.bind_host, opt.bind_port);
     let addr = addr.parse()?;
 
     let config = SchedulerConfig {
+        namespace: opt.namespace,
+        external_host: opt.external_host,

Review Comment:
   Sorry, this is slightly confusing :). It is still used to build the bind address for the server 
   
   ```
       let addr = format!("{}:{}", opt.bind_host, opt.bind_port);
       let addr = addr.parse()?;
   ```
   
   The reason `external_host` was added to `SchedulerConfig` is so that we can pass it in to `start_server` and build the `BallistaCluster` there only from the `SchedulerConfig`. In order to do that I had to pass a few more things from the `Config` struct generated by `configure_me` which isn't available in the lib code. 
   
   So really the only point is to further minimize what we do in `main.rs` to just setting up logging and validating CLI arguments



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-ballista] thinkharderdev commented on a diff in pull request #658: Cluster state refactor Part 2

Posted by "thinkharderdev (via GitHub)" <gi...@apache.org>.
thinkharderdev commented on code in PR #658:
URL: https://github.com/apache/arrow-ballista/pull/658#discussion_r1102912778


##########
ballista/scheduler/src/cluster/storage/sled.rs:
##########
@@ -305,33 +305,33 @@ mod tests {
         Ok(())
     }
 
-    #[tokio::test]
-    async fn multiple_operation() -> Result<(), Box<dyn std::error::Error>> {
-        let client = create_instance()?;
-        let key = "key".to_string();
-        let value = "value".as_bytes().to_vec();
-        let locks = client
-            .acquire_locks(vec![(Keyspace::ActiveJobs, ""), (Keyspace::Slots, "")])
-            .await?;
-
-        let _r: ballista_core::error::Result<()> = with_locks(locks, async {
-            let txn_ops = vec![
-                (Operation::Put(value.clone()), Keyspace::Slots, key.clone()),
-                (
-                    Operation::Put(value.clone()),
-                    Keyspace::ActiveJobs,
-                    key.clone(),
-                ),
-            ];
-            client.apply_txn(txn_ops).await?;
-            Ok(())
-        })
-        .await;
-
-        assert_eq!(client.get(Keyspace::Slots, key.as_str()).await?, value);
-        assert_eq!(client.get(Keyspace::ActiveJobs, key.as_str()).await?, value);
-        Ok(())
-    }
+    // #[tokio::test]

Review Comment:
   Sorry, oversight on my part. Commented while trying to get things to compile and forgot to uncomment. Fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-ballista] andygrove commented on a diff in pull request #658: Cluster state refactor Part 2

Posted by "andygrove (via GitHub)" <gi...@apache.org>.
andygrove commented on code in PR #658:
URL: https://github.com/apache/arrow-ballista/pull/658#discussion_r1102852483


##########
ballista/scheduler/src/bin/main.rs:
##########
@@ -117,10 +100,13 @@ async fn main() -> Result<()> {
             .init();
     }
 
-    let addr = format!("{bind_host}:{port}");
+    let addr = format!("{}:{}", opt.bind_host, opt.bind_port);
     let addr = addr.parse()?;
 
     let config = SchedulerConfig {
+        namespace: opt.namespace,
+        external_host: opt.external_host,

Review Comment:
   It looks like `bind_host` is not currently used in the scheduler, but it is in the executor, so this seems like an oversight.
   
   cc @avantgardnerio who may have opinions here



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-ballista] avantgardnerio commented on a diff in pull request #658: Cluster state refactor Part 2

Posted by "avantgardnerio (via GitHub)" <gi...@apache.org>.
avantgardnerio commented on code in PR #658:
URL: https://github.com/apache/arrow-ballista/pull/658#discussion_r1102919703


##########
ballista/scheduler/src/bin/main.rs:
##########
@@ -117,10 +100,13 @@ async fn main() -> Result<()> {
             .init();
     }
 
-    let addr = format!("{bind_host}:{port}");
+    let addr = format!("{}:{}", opt.bind_host, opt.bind_port);
     let addr = addr.parse()?;
 
     let config = SchedulerConfig {
+        namespace: opt.namespace,
+        external_host: opt.external_host,

Review Comment:
   FWIW, I believe `external_host` is required for FlightSQL.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-ballista] andygrove merged pull request #658: Cluster state refactor Part 2

Posted by "andygrove (via GitHub)" <gi...@apache.org>.
andygrove merged PR #658:
URL: https://github.com/apache/arrow-ballista/pull/658


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-ballista] Ted-Jiang commented on a diff in pull request #658: Cluster state refactor Part 2

Posted by "Ted-Jiang (via GitHub)" <gi...@apache.org>.
Ted-Jiang commented on code in PR #658:
URL: https://github.com/apache/arrow-ballista/pull/658#discussion_r1165095929


##########
ballista/scheduler/src/state/task_manager.rs:
##########
@@ -738,17 +639,18 @@ impl<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> TaskManager<T, U>
     }
 
     /// Get the `ExecutionGraph` for the given job ID from cache
-    pub(crate) async fn get_active_execution_graph(
+    pub(crate) fn get_active_execution_graph(
         &self,
         job_id: &str,
     ) -> Option<Arc<RwLock<ExecutionGraph>>> {
         self.active_job_cache
             .get(job_id)
-            .map(|value| value.execution_graph.clone())
+            .as_deref()
+            .map(|cached| cached.execution_graph.clone())
     }
 
     /// Remove the `ExecutionGraph` for the given job ID from cache
-    pub(crate) async fn remove_active_execution_graph(

Review Comment:
   @thinkharderdev  Sorry to interrupte, plz take look, could you plz tell me why remove the `async `, same in other functions 🤔 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org