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/01/08 04:49:37 UTC

[GitHub] [arrow-datafusion] yjshen commented on a change in pull request #1526: A simplified memory manager for query execution

yjshen commented on a change in pull request #1526:
URL: https://github.com/apache/arrow-datafusion/pull/1526#discussion_r780631979



##########
File path: datafusion/src/execution/memory_manager.rs
##########
@@ -0,0 +1,320 @@
+// 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.
+
+//! Manages all available memory during query execution
+
+use crate::error::Result;
+use async_trait::async_trait;
+use hashbrown::HashMap;
+use log::info;
+use std::fmt;
+use std::fmt::{Debug, Display, Formatter};
+use std::sync::atomic::{AtomicUsize, Ordering};
+use std::sync::{Arc, Mutex};
+use std::time::Duration;
+use tokio::task;
+use tokio::task::JoinHandle;
+
+static mut CONSUMER_ID: AtomicUsize = AtomicUsize::new(0);
+
+fn next_id() -> usize {
+    unsafe { CONSUMER_ID.fetch_add(1, Ordering::SeqCst) }
+}
+
+/// Type of the memory consumer
+pub enum ConsumerType {
+    /// consumers that can grow or shrink its memory usage during execution
+    /// such as spillable sorter, spillable hashmap, etc.
+    Controlling,
+    /// consumers that are not spillable, counting in for only tracking purpose.
+    Tracking,
+}
+
+#[derive(Clone, Debug, Hash, Eq, PartialEq)]
+/// Id that uniquely identifies a Memory Consumer
+pub struct MemoryConsumerId {
+    /// partition the consumer belongs to
+    pub partition_id: usize,
+    /// unique id
+    pub id: usize,
+}
+
+impl MemoryConsumerId {
+    /// Auto incremented new Id
+    pub fn new(partition_id: usize) -> Self {
+        let id = next_id();
+        Self { partition_id, id }
+    }
+}
+
+impl Display for MemoryConsumerId {
+    fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result {
+        write!(f, "{}:{}", self.partition_id, self.id)
+    }
+}
+
+#[async_trait]
+/// A memory consumer that either takes up memory (of type `ConsumerType::Tracking`)
+/// or grows/shrinks memory usage based on available memory (of type `ConsumerType::Controlling`).
+pub trait MemoryConsumer: Send + Sync + Debug {
+    /// Display name of the consumer
+    fn name(&self) -> String;
+
+    /// Unique id of the consumer
+    fn id(&self) -> &MemoryConsumerId;
+
+    /// Ptr to MemoryManager
+    fn memory_manager(&self) -> Arc<MemoryManager>;
+
+    /// Partition that the consumer belongs to
+    fn partition_id(&self) -> usize {
+        self.id().partition_id
+    }
+
+    /// Type of the consumer
+    fn type_(&self) -> &ConsumerType;
+
+    /// Grow memory by `required` to buffer more data in memory,
+    /// this may trigger spill before grow when the memory threshold is
+    /// reached for this consumer.
+    async fn try_grow(&self, required: usize) -> Result<()> {
+        let current_usage = self.mem_used();
+        let can_grow = self
+            .memory_manager()
+            .try_grow(required, current_usage, self.id())
+            .await;
+        if !can_grow {
+            info!(
+                "Failed to grow memory of {} from {}, spilling...",
+                human_readable_size(required),
+                self.id()
+            );
+            self.spill().await?;
+        }
+        Ok(())
+    }
+
+    /// Spill in-memory buffers to disk, free memory
+    async fn spill(&self) -> Result<()>;
+
+    /// Current memory used by this consumer
+    fn mem_used(&self) -> usize;
+
+    /// Current status of the consumer
+    fn str_repr(&self) -> String {
+        let mem = self.mem_used();
+        format!(
+            "{}[{}]: {}",
+            self.name(),
+            self.id(),
+            human_readable_size(mem)
+        )
+    }
+}
+
+/*
+The memory management architecture is the following:
+
+1. User designates max execution memory by setting RuntimeConfig.max_memory and RuntimeConfig.memory_fraction (float64 between 0..1).
+   The actual max memory DataFusion could use `pool_size =  max_memory * memory_fraction`.
+2. The entities that take up memory during its execution are called 'Memory Consumers'. Operators or others are encouraged to
+   register themselves to the memory manager and report its usage through `mem_used()`.
+3. There are two kinds of consumers:
+   - 'Controlling' consumers that would acquire memory during its execution and release memory through `spill` if no more memory is available.
+   - 'Tracking' consumers that exist for reporting purposes to provide a more accurate memory usage estimation for memory consumers.
+4. Controlling and tracking consumers share the pool. Each controlling consumer could acquire a maximum of
+   (pool_size - all_tracking_used) / active_num_controlling_consumers.
+
+            Memory Space for the DataFusion Lib / Process of `pool_size`
+   ┌──────────────────────────────────────────────z─────────────────────────────┐
+   │                                              z                             │
+   │                                              z                             │
+   │               Controlling                    z          Tracking           │
+   │            Memory Consumers                  z       Memory Consumers      │
+   │                                              z                             │
+   │                                              z                             │
+   └──────────────────────────────────────────────z─────────────────────────────┘
+*/
+
+/// Manage memory usage during physical plan execution
+pub struct MemoryManager {
+    consumers: Arc<Mutex<HashMap<MemoryConsumerId, Arc<dyn MemoryConsumer>>>>,
+    trackers: Arc<Mutex<HashMap<MemoryConsumerId, Arc<dyn MemoryConsumer>>>>,
+    trackers_total_usage: AtomicUsize,
+    pool_size: usize,
+    join_handle: Arc<Mutex<Option<JoinHandle<()>>>>,
+}
+
+impl MemoryManager {
+    /// Create new memory manager based on max available pool_size
+    pub fn new(pool_size: usize) -> Self {
+        info!(
+            "Creating memory manager with initial size {}",
+            human_readable_size(pool_size)
+        );
+        Self {
+            consumers: Arc::new(Mutex::new(HashMap::new())),
+            trackers: Arc::new(Mutex::new(HashMap::new())),
+            trackers_total_usage: AtomicUsize::new(0),
+            pool_size,
+            join_handle: Arc::new(Mutex::new(None)),
+        }
+    }
+
+    fn update_tracker_total(self: &Arc<Self>) {
+        let trackers = self.trackers.lock().unwrap();
+        if trackers.len() > 0 {
+            let sum = trackers.values().fold(0usize, |acc, y| acc + y.mem_used());
+            drop(trackers);
+            self.trackers_total_usage.store(sum, Ordering::SeqCst);
+        }
+    }
+
+    /// Initialize
+    pub(crate) fn initialize(self: &Arc<Self>) {
+        let manager = self.clone();
+        let handle = task::spawn(async move {

Review comment:
       Yes, I'm creating a background task that runs periodically to update `tracking consumers'` total memory usage, to avoid `controlling consumers` to ask for available memory frequently.




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