@@ -40,6 +40,7 @@ use log::warn;
4040pub struct CometUnifiedMemoryPool {
4141 task_memory_manager_handle : Arc < GlobalRef > ,
4242 used : AtomicUsize ,
43+ task_attempt_id : i64 ,
4344}
4445
4546impl Debug for CometUnifiedMemoryPool {
@@ -51,9 +52,13 @@ impl Debug for CometUnifiedMemoryPool {
5152}
5253
5354impl CometUnifiedMemoryPool {
54- pub fn new ( task_memory_manager_handle : Arc < GlobalRef > ) -> CometUnifiedMemoryPool {
55+ pub fn new (
56+ task_memory_manager_handle : Arc < GlobalRef > ,
57+ task_attempt_id : i64 ,
58+ ) -> CometUnifiedMemoryPool {
5559 Self {
5660 task_memory_manager_handle,
61+ task_attempt_id,
5762 used : AtomicUsize :: new ( 0 ) ,
5863 }
5964 }
@@ -82,7 +87,10 @@ impl Drop for CometUnifiedMemoryPool {
8287 fn drop ( & mut self ) {
8388 let used = self . used . load ( Relaxed ) ;
8489 if used != 0 {
85- warn ! ( "CometUnifiedMemoryPool dropped with {used} bytes still reserved" ) ;
90+ warn ! (
91+ "Task {} dropped CometUnifiedMemoryPool with {used} bytes still reserved" ,
92+ self . task_attempt_id
93+ ) ;
8694 }
8795 }
8896}
@@ -96,13 +104,20 @@ impl MemoryPool for CometUnifiedMemoryPool {
96104 }
97105
98106 fn shrink ( & self , _: & MemoryReservation , size : usize ) {
99- self . release_to_spark ( size)
100- . unwrap_or_else ( |_| panic ! ( "Failed to release {size} bytes" ) ) ;
107+ if let Err ( e) = self . release_to_spark ( size) {
108+ panic ! (
109+ "Task {} failed to return {size} bytes to Spark: {e:?}" ,
110+ self . task_attempt_id
111+ ) ;
112+ }
101113 if let Err ( prev) = self
102114 . used
103115 . fetch_update ( Relaxed , Relaxed , |old| old. checked_sub ( size) )
104116 {
105- panic ! ( "overflow when releasing {size} of {prev} bytes" ) ;
117+ panic ! (
118+ "Task {} overflow when releasing {size} of {prev} bytes" ,
119+ self . task_attempt_id
120+ ) ;
106121 }
107122 }
108123
@@ -116,7 +131,8 @@ impl MemoryPool for CometUnifiedMemoryPool {
116131 self . release_to_spark ( acquired as usize ) ?;
117132
118133 return Err ( resources_datafusion_err ! (
119- "Failed to acquire {} bytes, only got {}. Reserved: {}" ,
134+ "Task {} failed to acquire {} bytes, only got {}. Reserved: {}" ,
135+ self . task_attempt_id,
120136 additional,
121137 acquired,
122138 self . reserved( )
@@ -127,7 +143,8 @@ impl MemoryPool for CometUnifiedMemoryPool {
127143 . fetch_update ( Relaxed , Relaxed , |old| old. checked_add ( acquired as usize ) )
128144 {
129145 return Err ( resources_datafusion_err ! (
130- "Failed to acquire {} bytes due to overflow. Reserved: {}" ,
146+ "Task {} failed to acquire {} bytes due to overflow. Reserved: {}" ,
147+ self . task_attempt_id,
131148 additional,
132149 prev
133150 ) ) ;
0 commit comments