You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2022/11/18 11:00:28 UTC

[GitHub] [arrow-ballista] thinkharderdev commented on a diff in pull request #523: Provide a memory StateBackendClient

thinkharderdev commented on code in PR #523:
URL: https://github.com/apache/arrow-ballista/pull/523#discussion_r1026287065


##########
ballista/scheduler/src/state/backend/memory.rs:
##########
@@ -0,0 +1,395 @@
+// 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.
+
+use crate::state::backend::utils::sled_subscriber::{Subscriber, Subscribers};
+use crate::state::backend::{
+    Keyspace, Lock, Operation, StateBackendClient, Watch, WatchEvent,
+};
+use ballista_core::error::Result;
+use dashmap::DashMap;
+use futures::{FutureExt, Stream};
+use log::warn;
+use std::collections::{BTreeMap, HashSet};
+use std::sync::Arc;
+use tokio::sync::Mutex;
+
+type KeySpaceState = BTreeMap<String, Vec<u8>>;
+type KeyLock = Arc<Mutex<()>>;
+
+/// A [`StateBackendClient`] implementation that uses in memory map to save cluster state.
+#[derive(Clone, Default)]
+pub struct MemoryBackendClient {
+    /// The key is the KeySpace. For every KeySpace, there will be a tree map which is better for prefix filtering
+    states: DashMap<String, KeySpaceState>,
+    /// The key is the full key formatted like "/KeySpace/key". It's a flatted map
+    locks: DashMap<String, KeyLock>,
+    subscribers: Arc<Subscribers>,
+}
+
+impl MemoryBackendClient {
+    pub fn new() -> Self {
+        Self::default()
+    }
+
+    fn get_space_key(keyspace: &Keyspace) -> String {
+        format!("/{:?}", keyspace)
+    }
+
+    fn get_flat_key(keyspace: &Keyspace, key: &str) -> String {
+        format!("/{:?}/{}", keyspace, key)
+    }
+}
+
+#[tonic::async_trait]
+impl StateBackendClient for MemoryBackendClient {
+    async fn get(&self, keyspace: Keyspace, key: &str) -> Result<Vec<u8>> {
+        let space_key = Self::get_space_key(&keyspace);
+        Ok(self
+            .states
+            .get(&space_key)
+            .map(|space_state| space_state.value().get(key).cloned().unwrap_or_default())
+            .unwrap_or_default())
+    }
+
+    async fn get_from_prefix(
+        &self,
+        keyspace: Keyspace,
+        prefix: &str,
+    ) -> Result<Vec<(String, Vec<u8>)>> {
+        let space_key = Self::get_space_key(&keyspace);
+        Ok(self
+            .states
+            .get(&space_key)
+            .map(|space_state| {
+                space_state
+                    .value()
+                    .range(prefix.to_owned()..)
+                    .take_while(|(k, _)| k.starts_with(prefix))
+                    .map(|e| (format!("{}/{}", space_key, e.0), e.1.clone()))
+                    .collect()
+            })
+            .unwrap_or_default())
+    }
+
+    async fn scan(
+        &self,
+        keyspace: Keyspace,
+        limit: Option<usize>,
+    ) -> Result<Vec<(String, Vec<u8>)>> {
+        let space_key = Self::get_space_key(&keyspace);
+        Ok(self
+            .states
+            .get(&space_key)
+            .map(|space_state| {
+                if let Some(limit) = limit {
+                    space_state
+                        .value()
+                        .iter()
+                        .take(limit)
+                        .map(|e| (format!("{}/{}", space_key, e.0), e.1.clone()))
+                        .collect::<Vec<(String, Vec<u8>)>>()
+                } else {
+                    space_state
+                        .value()
+                        .iter()
+                        .map(|e| (format!("{}/{}", space_key, e.0), e.1.clone()))
+                        .collect::<Vec<(String, Vec<u8>)>>()
+                }
+            })
+            .unwrap_or_default())
+    }
+
+    async fn scan_keys(&self, keyspace: Keyspace) -> Result<HashSet<String>> {
+        let space_key = Self::get_space_key(&keyspace);
+        Ok(self
+            .states
+            .get(&space_key)
+            .map(|space_state| {
+                space_state
+                    .value()
+                    .iter()
+                    .map(|e| format!("{}/{}", space_key, e.0))
+                    .collect::<HashSet<String>>()
+            })
+            .unwrap_or_default())
+    }
+
+    async fn put(&self, keyspace: Keyspace, key: String, value: Vec<u8>) -> Result<()> {
+        let space_key = Self::get_space_key(&keyspace);
+        if !self.states.contains_key(&space_key) {
+            self.states.insert(space_key.clone(), BTreeMap::default());
+        }
+        self.states
+            .get_mut(&space_key)
+            .unwrap()
+            .value_mut()
+            .insert(key.clone(), value.clone());
+
+        // Notify subscribers
+        let full_key = format!("{}/{}", space_key, key);
+        if let Some(res) = self.subscribers.reserve(&full_key) {
+            let event = WatchEvent::Put(full_key, value);
+            res.complete(&event);
+        }
+
+        Ok(())
+    }
+
+    async fn apply_txn(&self, ops: Vec<(Operation, Keyspace, String)>) -> Result<()> {
+        let mut locks = vec![];
+        for (_, keyspace, key) in ops.iter() {
+            locks.push(self.lock(keyspace.clone(), key).await?);
+        }
+
+        for (op, keyspace, key) in ops.into_iter() {
+            match op {
+                Operation::Delete => {
+                    self.delete(keyspace, &key).await?;
+                }
+                Operation::Put(value) => {
+                    self.put(keyspace, key, value).await?;
+                }
+            };
+        }
+
+        Ok(())
+    }
+
+    async fn mv(
+        &self,
+        from_keyspace: Keyspace,
+        to_keyspace: Keyspace,
+        key: &str,
+    ) -> Result<()> {
+        let from_space_key = Self::get_space_key(&from_keyspace);
+
+        if let Some(from_space_state) = self.states.get(&from_space_key) {
+            if let Some(state) = from_space_state.value().get(key) {
+                let ops = vec![
+                    (Operation::Delete, from_keyspace, key.to_owned()),
+                    (Operation::Put(state.clone()), to_keyspace, key.to_owned()),
+                ];
+                self.apply_txn(ops).await
+            } else {
+                // TODO should this return an error?

Review Comment:
   I think this should be consistent with the Sled and Etcd backends which I believe would both return an error in this scenario



##########
ballista/scheduler/src/state/backend/utils/mod.rs:
##########
@@ -0,0 +1,21 @@
+// 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.
+
+#[allow(dead_code)]
+mod sled_oneshot;
+#[allow(dead_code)]
+pub(crate) mod sled_subscriber;

Review Comment:
   Confused about the naming here. Why is this called `sled_subscriber`? It's used by the memory backend right? 



##########
ballista/scheduler/src/state/backend/memory.rs:
##########
@@ -0,0 +1,395 @@
+// 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.
+
+use crate::state::backend::utils::sled_subscriber::{Subscriber, Subscribers};
+use crate::state::backend::{
+    Keyspace, Lock, Operation, StateBackendClient, Watch, WatchEvent,
+};
+use ballista_core::error::Result;
+use dashmap::DashMap;
+use futures::{FutureExt, Stream};
+use log::warn;
+use std::collections::{BTreeMap, HashSet};
+use std::sync::Arc;
+use tokio::sync::Mutex;
+
+type KeySpaceState = BTreeMap<String, Vec<u8>>;
+type KeyLock = Arc<Mutex<()>>;
+
+/// A [`StateBackendClient`] implementation that uses in memory map to save cluster state.
+#[derive(Clone, Default)]
+pub struct MemoryBackendClient {
+    /// The key is the KeySpace. For every KeySpace, there will be a tree map which is better for prefix filtering
+    states: DashMap<String, KeySpaceState>,
+    /// The key is the full key formatted like "/KeySpace/key". It's a flatted map
+    locks: DashMap<String, KeyLock>,
+    subscribers: Arc<Subscribers>,
+}
+
+impl MemoryBackendClient {
+    pub fn new() -> Self {
+        Self::default()
+    }
+
+    fn get_space_key(keyspace: &Keyspace) -> String {
+        format!("/{:?}", keyspace)
+    }
+
+    fn get_flat_key(keyspace: &Keyspace, key: &str) -> String {
+        format!("/{:?}/{}", keyspace, key)
+    }
+}
+
+#[tonic::async_trait]
+impl StateBackendClient for MemoryBackendClient {
+    async fn get(&self, keyspace: Keyspace, key: &str) -> Result<Vec<u8>> {
+        let space_key = Self::get_space_key(&keyspace);
+        Ok(self
+            .states
+            .get(&space_key)
+            .map(|space_state| space_state.value().get(key).cloned().unwrap_or_default())
+            .unwrap_or_default())
+    }
+
+    async fn get_from_prefix(
+        &self,
+        keyspace: Keyspace,
+        prefix: &str,
+    ) -> Result<Vec<(String, Vec<u8>)>> {
+        let space_key = Self::get_space_key(&keyspace);
+        Ok(self
+            .states
+            .get(&space_key)
+            .map(|space_state| {
+                space_state
+                    .value()
+                    .range(prefix.to_owned()..)
+                    .take_while(|(k, _)| k.starts_with(prefix))
+                    .map(|e| (format!("{}/{}", space_key, e.0), e.1.clone()))
+                    .collect()
+            })
+            .unwrap_or_default())
+    }
+
+    async fn scan(
+        &self,
+        keyspace: Keyspace,
+        limit: Option<usize>,
+    ) -> Result<Vec<(String, Vec<u8>)>> {
+        let space_key = Self::get_space_key(&keyspace);
+        Ok(self
+            .states
+            .get(&space_key)
+            .map(|space_state| {
+                if let Some(limit) = limit {
+                    space_state
+                        .value()
+                        .iter()
+                        .take(limit)
+                        .map(|e| (format!("{}/{}", space_key, e.0), e.1.clone()))
+                        .collect::<Vec<(String, Vec<u8>)>>()
+                } else {
+                    space_state
+                        .value()
+                        .iter()
+                        .map(|e| (format!("{}/{}", space_key, e.0), e.1.clone()))
+                        .collect::<Vec<(String, Vec<u8>)>>()
+                }
+            })
+            .unwrap_or_default())
+    }
+
+    async fn scan_keys(&self, keyspace: Keyspace) -> Result<HashSet<String>> {
+        let space_key = Self::get_space_key(&keyspace);
+        Ok(self
+            .states
+            .get(&space_key)
+            .map(|space_state| {
+                space_state
+                    .value()
+                    .iter()
+                    .map(|e| format!("{}/{}", space_key, e.0))
+                    .collect::<HashSet<String>>()
+            })
+            .unwrap_or_default())
+    }
+
+    async fn put(&self, keyspace: Keyspace, key: String, value: Vec<u8>) -> Result<()> {
+        let space_key = Self::get_space_key(&keyspace);
+        if !self.states.contains_key(&space_key) {
+            self.states.insert(space_key.clone(), BTreeMap::default());
+        }
+        self.states
+            .get_mut(&space_key)
+            .unwrap()
+            .value_mut()
+            .insert(key.clone(), value.clone());
+
+        // Notify subscribers
+        let full_key = format!("{}/{}", space_key, key);
+        if let Some(res) = self.subscribers.reserve(&full_key) {
+            let event = WatchEvent::Put(full_key, value);
+            res.complete(&event);
+        }
+
+        Ok(())
+    }
+
+    async fn apply_txn(&self, ops: Vec<(Operation, Keyspace, String)>) -> Result<()> {
+        let mut locks = vec![];
+        for (_, keyspace, key) in ops.iter() {
+            locks.push(self.lock(keyspace.clone(), key).await?);
+        }
+
+        for (op, keyspace, key) in ops.into_iter() {
+            match op {
+                Operation::Delete => {
+                    self.delete(keyspace, &key).await?;
+                }
+                Operation::Put(value) => {
+                    self.put(keyspace, key, value).await?;
+                }
+            };
+        }
+
+        Ok(())
+    }
+
+    async fn mv(
+        &self,
+        from_keyspace: Keyspace,
+        to_keyspace: Keyspace,
+        key: &str,
+    ) -> Result<()> {
+        let from_space_key = Self::get_space_key(&from_keyspace);
+
+        if let Some(from_space_state) = self.states.get(&from_space_key) {
+            if let Some(state) = from_space_state.value().get(key) {
+                let ops = vec![
+                    (Operation::Delete, from_keyspace, key.to_owned()),
+                    (Operation::Put(state.clone()), to_keyspace, key.to_owned()),
+                ];
+                self.apply_txn(ops).await
+            } else {
+                // TODO should this return an error?
+                warn!(
+                    "Cannot move value at {}/{}, does not exist",
+                    from_space_key, key
+                );
+                Ok(())
+            }
+        } else {
+            // TODO should this return an error?
+            warn!(
+                "Cannot move value at {}/{}, does not exist",
+                from_space_key, key
+            );
+            Ok(())
+        }
+    }
+
+    async fn lock(&self, keyspace: Keyspace, key: &str) -> Result<Box<dyn Lock>> {
+        let flat_key = Self::get_flat_key(&keyspace, key);
+        if let Some(lock) = self.locks.get_mut(&flat_key) {
+            Ok(Box::new(lock.value().clone().lock_owned().await))
+        } else {
+            let new_lock = Arc::new(Mutex::new(()));
+            self.locks.insert(flat_key, new_lock.clone());
+            Ok(Box::new(new_lock.lock_owned().await))
+        }
+    }

Review Comment:
   Hmm, I think this might allow multiple locks. 
   
   Can we use the entry API here? 
   
   ```rust
   let new_lock = Arc::new(Mutex::new(()));
   
   let lock = self.locks.entry(&flat_key).or_insert_with(|| Arc::new(Mutex::new(())));
   Ok(Box::new(lock.value().clone().lock_owned().await))
   ```



-- 
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