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

agrove 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 e86ad26  Add option param for standalone mode (#42)
e86ad26 is described below

commit e86ad26a678a32e2743bc031ae5cb81e6931a231
Author: K.I. (Dennis) Jung <[email protected]>
AuthorDate: Tue Apr 27 13:33:55 2021 +0900

    Add option param for standalone mode (#42)
    
    * add option param for standalone mode
    
    * Run formatter
---
 ballista/rust/executor/executor_config_spec.toml |  7 ++++++-
 ballista/rust/executor/src/main.rs               | 10 ++++++++--
 2 files changed, 14 insertions(+), 3 deletions(-)

diff --git a/ballista/rust/executor/executor_config_spec.toml 
b/ballista/rust/executor/executor_config_spec.toml
index cb47ca0..f7c2f03 100644
--- a/ballista/rust/executor/executor_config_spec.toml
+++ b/ballista/rust/executor/executor_config_spec.toml
@@ -76,4 +76,9 @@ abbr = "c"
 name = "concurrent_tasks"
 type = "usize"
 default = "4"
-doc = "Max concurrent tasks."
\ No newline at end of file
+doc = "Max concurrent tasks."
+
+[[param]]
+name = "scheduler_data_path"
+type = "String"
+doc = "Path for standalone data"
diff --git a/ballista/rust/executor/src/main.rs 
b/ballista/rust/executor/src/main.rs
index 2718ea3..e620fa4 100644
--- a/ballista/rust/executor/src/main.rs
+++ b/ballista/rust/executor/src/main.rs
@@ -107,8 +107,14 @@ async fn main() -> Result<()> {
 
     if opt.local {
         info!("Running in local mode. Scheduler will be run in-proc");
-        let client = StandaloneClient::try_new_temporary()
-            .context("Could not create standalone config backend")?;
+
+        let client = match opt.scheduler_data_path {
+            Some(v) => StandaloneClient::try_new(v)
+                .context("Could not create standalone config backend")?,
+            None => StandaloneClient::try_new_temporary()
+                .context("Could not create standalone config backend")?,
+        };
+
         let server =
             SchedulerGrpcServer::new(SchedulerServer::new(Arc::new(client), 
namespace));
         let addr = format!("{}:{}", bind_host, scheduler_port);

Reply via email to