1818pub mod memory_pool;
1919
2020use crate :: error:: DataFusionError :: OutOfMemory ;
21- use crate :: error:: { DataFusionError , Result } ;
22- use crate :: execution:: disk_manager:: DiskManager ;
21+ use crate :: error:: Result ;
2322use crate :: execution:: memory_management:: memory_pool:: {
2423 ConstraintExecutionMemoryPool , DummyExecutionMemoryPool , ExecutionMemoryPool ,
2524} ;
2625use async_trait:: async_trait;
2726use hashbrown:: { HashMap , HashSet } ;
2827use log:: { debug, info} ;
2928use std:: fmt;
30- use std:: fmt:: { Display , Formatter } ;
29+ use std:: fmt:: { Display , Formatter , Debug } ;
3130use std:: sync:: atomic:: { AtomicUsize , Ordering } ;
3231use std:: sync:: { Arc , Mutex } ;
32+ use std:: borrow:: BorrowMut ;
3333
3434static mut CONSUMER_ID : AtomicUsize = AtomicUsize :: new ( 0 ) ;
3535
@@ -41,21 +41,21 @@ pub struct MemoryManager {
4141
4242impl MemoryManager {
4343 pub fn new ( exec_pool_size : usize ) -> Self {
44- let pool : dyn ExecutionMemoryPool = if exec_pool_size == usize:: MAX {
45- DummyExecutionMemoryPool :: new ( )
44+ let execution_pool = if exec_pool_size == usize:: MAX {
45+ Arc :: new ( DummyExecutionMemoryPool :: new ( ) as dyn ExecutionMemoryPool )
4646 } else {
47- ConstraintExecutionMemoryPool :: new ( exec_pool_size)
47+ Arc :: new ( ConstraintExecutionMemoryPool :: new ( exec_pool_size) )
4848 } ;
4949 Self {
50- execution_pool : Arc :: new ( pool ) ,
50+ execution_pool,
5151 partition_memory_manager : Arc :: new ( Mutex :: new ( HashMap :: new ( ) ) ) ,
5252 }
5353 }
5454
5555 pub fn acquire_exec_memory (
5656 self : Arc < Self > ,
5757 required : usize ,
58- consumer : & dyn MemoryConsumer ,
58+ consumer : Arc < dyn MemoryConsumer > ,
5959 ) -> Result < usize > {
6060 let partition_id = consumer. partition_id ( ) ;
6161 let partition_manager = {
@@ -70,7 +70,7 @@ impl MemoryManager {
7070 pub fn acquire_exec_pool_memory (
7171 & self ,
7272 required : usize ,
73- consumer : & dyn MemoryConsumer ,
73+ consumer : & Arc < dyn MemoryConsumer > ,
7474 ) -> usize {
7575 self . execution_pool . acquire_memory ( required, consumer)
7676 }
@@ -110,7 +110,7 @@ fn next_id() -> usize {
110110pub struct PartitionMemoryManager {
111111 memory_manager : Arc < MemoryManager > ,
112112 partition_id : usize ,
113- consumers : Arc < Mutex < HashSet < dyn MemoryConsumer > > > ,
113+ consumers : Arc < Mutex < HashSet < Arc < dyn MemoryConsumer > > > > ,
114114}
115115
116116impl PartitionMemoryManager {
@@ -125,12 +125,12 @@ impl PartitionMemoryManager {
125125 pub fn acquire_exec_memory (
126126 & mut self ,
127127 required : usize ,
128- consumer : & dyn MemoryConsumer ,
128+ consumer : Arc < dyn MemoryConsumer > ,
129129 ) -> Result < usize > {
130- let mut consumers = self . consumers . lock ( ) . unwrap ( ) ;
130+ let mut consumers = self . consumers . lock ( ) . unwrap ( ) . borrow_mut ( ) ;
131131 let mut got = self
132132 . memory_manager
133- . acquire_exec_pool_memory ( required, consumer) ;
133+ . acquire_exec_pool_memory ( required, & consumer) ;
134134 if got < required {
135135 // spill others first
136136 }
@@ -140,8 +140,7 @@ impl PartitionMemoryManager {
140140 }
141141
142142 if got < required {
143- return Err ( OutOfMemory ( format ! (
144- "Unable to acquire {} bytes of memory, got {}" ,
143+ return Err ( OutOfMemory ( format ! ( "Unable to acquire {} bytes of memory, got {}" ,
145144 required, got
146145 ) ) ) ;
147146 }
@@ -162,14 +161,14 @@ impl PartitionMemoryManager {
162161 info ! (
163162 "Consumer {} acquired {}" ,
164163 c. str_repr( ) ,
165- human_readable_size( cur_used)
164+ human_readable_size( cur_used as usize )
166165 )
167166 }
168167 }
169168 let no_consumer_size = self
170169 . memory_manager
171170 . exec_memory_used_for_partition ( self . partition_id )
172- - used;
171+ - ( used as usize ) ;
173172 info ! (
174173 "{} bytes of memory were used for partition {} without specific consumer" ,
175174 human_readable_size( no_consumer_size) ,
@@ -178,10 +177,10 @@ impl PartitionMemoryManager {
178177 }
179178}
180179
181- #[ derive( Debug , Clone ) ]
180+ #[ derive( Clone , Debug ) ]
182181pub struct MemoryConsumerId {
183- partition_id : usize ,
184- id : usize ,
182+ pub partition_id : usize ,
183+ pub id : usize ,
185184}
186185
187186impl MemoryConsumerId {
@@ -198,20 +197,20 @@ impl Display for MemoryConsumerId {
198197}
199198
200199#[ async_trait]
201- pub trait MemoryConsumer {
200+ pub trait MemoryConsumer : Send + Sync + Debug {
202201 /// Display name of the consumer
203202 fn name ( & self ) -> String ;
204203 /// Unique id of the consumer
205204 fn id ( & self ) -> & MemoryConsumerId ;
206205
207206 fn memory_manager ( & self ) -> Arc < MemoryManager > ;
208207 /// partition that the consumer belongs to
209- fn partition_id ( & self ) -> uszie {
208+ fn partition_id ( & self ) -> usize {
210209 self . id ( ) . partition_id
211210 }
212211 /// Try allocate `required` bytes as needed
213- fn allocate ( & self , required : usize ) -> Result < ( ) > {
214- let got = self . memory_manager ( ) . acquire_exec_memory ( required, self ) ?;
212+ fn allocate ( self : Arc < Self > , required : usize ) -> Result < ( ) > {
213+ let got = self . memory_manager ( ) . acquire_exec_memory ( required, self . clone ( ) ) ?;
215214 self . update_used ( got as isize ) ;
216215 Ok ( ( ) )
217216 }
@@ -250,15 +249,15 @@ fn human_readable_size(size: usize) -> String {
250249 let size = size as u64 ;
251250 let ( value, unit) = {
252251 if size >= 2 * TB {
253- ( size as f64 / TB , "TB" )
252+ ( size as f64 / TB as f64 , "TB" )
254253 } else if size >= 2 * GB {
255- ( size as f64 / GB , "GB" )
254+ ( size as f64 / GB as f64 , "GB" )
256255 } else if size >= 2 * MB {
257- ( size as f64 / MB , "MB" )
256+ ( size as f64 / MB as f64 , "MB" )
258257 } else if size >= 2 * KB {
259- ( size as f64 / KB , "KB" )
258+ ( size as f64 / KB as f64 , "KB" )
260259 } else {
261- ( size, "B" )
260+ ( size as f64 , "B" )
262261 }
263262 } ;
264263 format ! ( "{:.1} {}" , value, unit)
0 commit comments