-
Notifications
You must be signed in to change notification settings - Fork 267
minor: include taskAttemptId in log messages #2467
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
ac621aa
ddd268b
93e2517
abf2e89
ca4aa67
2b7a2c5
453feb4
5f9edda
aeb41c2
b49f4d6
7a4aeb4
d3218f9
edcead7
3cda8de
37f76eb
ab8731d
2d882d2
439b5dd
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change | ||||
|---|---|---|---|---|---|---|
|
|
@@ -40,6 +40,7 @@ use log::warn; | |||||
| pub struct CometUnifiedMemoryPool { | ||||||
| task_memory_manager_handle: Arc<GlobalRef>, | ||||||
| used: AtomicUsize, | ||||||
| task_attempt_id: i64, | ||||||
| } | ||||||
|
|
||||||
| impl Debug for CometUnifiedMemoryPool { | ||||||
|
|
@@ -51,9 +52,13 @@ impl Debug for CometUnifiedMemoryPool { | |||||
| } | ||||||
|
|
||||||
| impl CometUnifiedMemoryPool { | ||||||
| pub fn new(task_memory_manager_handle: Arc<GlobalRef>) -> CometUnifiedMemoryPool { | ||||||
| pub fn new( | ||||||
| task_memory_manager_handle: Arc<GlobalRef>, | ||||||
| task_attempt_id: i64, | ||||||
| ) -> CometUnifiedMemoryPool { | ||||||
| Self { | ||||||
| task_memory_manager_handle, | ||||||
| task_attempt_id, | ||||||
| used: AtomicUsize::new(0), | ||||||
| } | ||||||
| } | ||||||
|
|
@@ -82,7 +87,10 @@ impl Drop for CometUnifiedMemoryPool { | |||||
| fn drop(&mut self) { | ||||||
| let used = self.used.load(Relaxed); | ||||||
| if used != 0 { | ||||||
| warn!("CometUnifiedMemoryPool dropped with {used} bytes still reserved"); | ||||||
| warn!( | ||||||
| "Task {} dropped CometUnifiedMemoryPool with {used} bytes still reserved", | ||||||
| self.task_attempt_id | ||||||
| ); | ||||||
| } | ||||||
| } | ||||||
| } | ||||||
|
|
@@ -96,13 +104,20 @@ impl MemoryPool for CometUnifiedMemoryPool { | |||||
| } | ||||||
|
|
||||||
| fn shrink(&self, _: &MemoryReservation, size: usize) { | ||||||
| self.release_to_spark(size) | ||||||
| .unwrap_or_else(|_| panic!("Failed to release {size} bytes")); | ||||||
| if let Err(e) = self.release_to_spark(size) { | ||||||
| panic!( | ||||||
| "Task {} failed to return {size} bytes to Spark: {e:?}", | ||||||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Suggested change
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I was following Spark's convention for logging so that we can use grep to search for
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Sounds good! |
||||||
| self.task_attempt_id | ||||||
| ); | ||||||
| } | ||||||
| if let Err(prev) = self | ||||||
| .used | ||||||
| .fetch_update(Relaxed, Relaxed, |old| old.checked_sub(size)) | ||||||
| { | ||||||
| panic!("overflow when releasing {size} of {prev} bytes"); | ||||||
| panic!( | ||||||
| "Task {} overflow when releasing {size} of {prev} bytes", | ||||||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Suggested change
|
||||||
| self.task_attempt_id | ||||||
| ); | ||||||
| } | ||||||
| } | ||||||
|
|
||||||
|
|
@@ -116,7 +131,8 @@ impl MemoryPool for CometUnifiedMemoryPool { | |||||
| self.release_to_spark(acquired as usize)?; | ||||||
|
|
||||||
| return Err(resources_datafusion_err!( | ||||||
| "Failed to acquire {} bytes, only got {}. Reserved: {}", | ||||||
| "Task {} failed to acquire {} bytes, only got {}. Reserved: {}", | ||||||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Suggested change
|
||||||
| self.task_attempt_id, | ||||||
| additional, | ||||||
| acquired, | ||||||
| self.reserved() | ||||||
|
|
@@ -127,7 +143,8 @@ impl MemoryPool for CometUnifiedMemoryPool { | |||||
| .fetch_update(Relaxed, Relaxed, |old| old.checked_add(acquired as usize)) | ||||||
| { | ||||||
| return Err(resources_datafusion_err!( | ||||||
| "Failed to acquire {} bytes due to overflow. Reserved: {}", | ||||||
| "Task {} failed to acquire {} bytes due to overflow. Reserved: {}", | ||||||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Suggested change
|
||||||
| self.task_attempt_id, | ||||||
| additional, | ||||||
| prev | ||||||
| )); | ||||||
|
|
||||||
| Original file line number | Diff line number | Diff line change | ||||
|---|---|---|---|---|---|---|
|
|
@@ -40,22 +40,36 @@ public class CometTaskMemoryManager { | |||||
| /** The id uniquely identifies the native plan this memory manager is associated to */ | ||||||
| private final long id; | ||||||
|
|
||||||
| private final long taskAttemptId; | ||||||
|
|
||||||
| public final TaskMemoryManager internal; | ||||||
| private final NativeMemoryConsumer nativeMemoryConsumer; | ||||||
| private final AtomicLong used = new AtomicLong(); | ||||||
|
|
||||||
| public CometTaskMemoryManager(long id) { | ||||||
| public CometTaskMemoryManager(long id, long taskAttemptId) { | ||||||
| this.id = id; | ||||||
| this.taskAttemptId = taskAttemptId; | ||||||
| this.internal = TaskContext$.MODULE$.get().taskMemoryManager(); | ||||||
| this.nativeMemoryConsumer = new NativeMemoryConsumer(); | ||||||
| } | ||||||
|
|
||||||
| // Called by Comet native through JNI. | ||||||
| // Returns the actual amount of memory (in bytes) granted. | ||||||
| public long acquireMemory(long size) { | ||||||
| if (logger.isTraceEnabled()) { | ||||||
| logger.trace("Task {} requested {} bytes", taskAttemptId, size); | ||||||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Suggested change
|
||||||
| } | ||||||
| long acquired = internal.acquireExecutionMemory(size, nativeMemoryConsumer); | ||||||
| used.addAndGet(acquired); | ||||||
| long newUsed = used.addAndGet(acquired); | ||||||
| if (acquired < size) { | ||||||
| logger.warn( | ||||||
| "Task {} requested {} bytes but only received {} bytes. Current allocation is {} and " | ||||||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Suggested change
|
||||||
| + "the total memory consumption is {} bytes.", | ||||||
| taskAttemptId, | ||||||
| size, | ||||||
| acquired, | ||||||
| newUsed, | ||||||
| internal.getMemoryConsumptionForThisTask()); | ||||||
| // If memory manager is not able to acquire the requested size, log memory usage | ||||||
| internal.showMemoryUsage(); | ||||||
| } | ||||||
|
|
@@ -64,10 +78,16 @@ public long acquireMemory(long size) { | |||||
|
|
||||||
| // Called by Comet native through JNI | ||||||
| public void releaseMemory(long size) { | ||||||
| if (logger.isTraceEnabled()) { | ||||||
| logger.trace("Task {} released {} bytes", taskAttemptId, size); | ||||||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Suggested change
|
||||||
| } | ||||||
| long newUsed = used.addAndGet(-size); | ||||||
| if (newUsed < 0) { | ||||||
| logger.error( | ||||||
| "Used memory is negative: " + newUsed + " after releasing memory chunk of: " + size); | ||||||
| "Task {} used memory is negative ({}) after releasing {} bytes", | ||||||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Suggested change
|
||||||
| taskAttemptId, | ||||||
| newUsed, | ||||||
| size); | ||||||
| } | ||||||
| internal.releaseExecutionMemory(size, nativeMemoryConsumer); | ||||||
| } | ||||||
|
|
||||||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.