-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-48134][CORE] Spark core (java side): Migrate error/warn/info with variables to structured logging framework
#46390
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
fda2a95
f97c2a6
442691f
4f988f1
e7a7b51
a97a1f3
4943ec3
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 |
|---|---|---|
|
|
@@ -13,13 +13,6 @@ | |
| */ | ||
| package org.apache.spark.io; | ||
|
|
||
| import com.google.common.base.Preconditions; | ||
|
Contributor
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. Taking this chance, let's adjust the |
||
| import com.google.common.base.Throwables; | ||
| import org.apache.spark.util.ThreadUtils; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| import javax.annotation.concurrent.GuardedBy; | ||
| import java.io.EOFException; | ||
| import java.io.IOException; | ||
| import java.io.InputStream; | ||
|
|
@@ -30,6 +23,16 @@ | |
| import java.util.concurrent.atomic.AtomicBoolean; | ||
| import java.util.concurrent.locks.Condition; | ||
| import java.util.concurrent.locks.ReentrantLock; | ||
| import javax.annotation.concurrent.GuardedBy; | ||
|
|
||
| import com.google.common.base.Preconditions; | ||
| import com.google.common.base.Throwables; | ||
|
|
||
| import org.apache.spark.internal.Logger; | ||
| import org.apache.spark.internal.LoggerFactory; | ||
| import org.apache.spark.internal.LogKeys; | ||
| import org.apache.spark.internal.MDC; | ||
| import org.apache.spark.util.ThreadUtils; | ||
|
|
||
| /** | ||
| * {@link InputStream} implementation which asynchronously reads ahead from the underlying input | ||
|
|
@@ -205,7 +208,7 @@ private void closeUnderlyingInputStreamIfNecessary() { | |
| try { | ||
| underlyingInputStream.close(); | ||
| } catch (IOException e) { | ||
| logger.warn(e.getMessage(), e); | ||
| logger.warn("{}", e, MDC.of(LogKeys.ERROR$.MODULE$, e.getMessage())); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -23,17 +23,19 @@ | |
| import java.util.LinkedList; | ||
| import java.util.zip.Checksum; | ||
|
|
||
| import org.apache.spark.SparkException; | ||
| import scala.Tuple2; | ||
|
|
||
| import com.google.common.annotations.VisibleForTesting; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| import org.apache.spark.SparkConf; | ||
| import org.apache.spark.SparkException; | ||
| import org.apache.spark.TaskContext; | ||
| import org.apache.spark.executor.ShuffleWriteMetrics; | ||
| import org.apache.spark.internal.config.package$; | ||
| import org.apache.spark.internal.Logger; | ||
| import org.apache.spark.internal.LoggerFactory; | ||
| import org.apache.spark.internal.LogKeys; | ||
| import org.apache.spark.internal.MDC; | ||
| import org.apache.spark.memory.MemoryConsumer; | ||
| import org.apache.spark.memory.SparkOutOfMemoryError; | ||
| import org.apache.spark.memory.TaskMemoryManager; | ||
|
|
@@ -159,11 +161,11 @@ private void writeSortedFile(boolean isFinalFile) { | |
| if (!isFinalFile) { | ||
| logger.info( | ||
| "Task {} on Thread {} spilling sort data of {} to disk ({} {} so far)", | ||
| taskContext.taskAttemptId(), | ||
| Thread.currentThread().getId(), | ||
| Utils.bytesToString(getMemoryUsage()), | ||
| spills.size(), | ||
| spills.size() != 1 ? " times" : " time"); | ||
| MDC.of(LogKeys.TASK_ATTEMPT_ID$.MODULE$, taskContext.taskAttemptId()), | ||
| MDC.of(LogKeys.THREAD_ID$.MODULE$, Thread.currentThread().getId()), | ||
| MDC.of(LogKeys.MEMORY_SIZE$.MODULE$, Utils.bytesToString(getMemoryUsage())), | ||
| MDC.of(LogKeys.NUM_SPILL_INFOS$.MODULE$, spills.size()), | ||
| MDC.of(LogKeys.SPILL_TIMES$.MODULE$, spills.size() != 1 ? "times" : "time")); | ||
| } | ||
|
|
||
| // This call performs the actual sort. | ||
|
|
@@ -349,7 +351,8 @@ public void cleanupResources() { | |
| } | ||
| for (SpillInfo spill : spills) { | ||
| if (spill.file.exists() && !spill.file.delete()) { | ||
| logger.error("Unable to delete spill file {}", spill.file.getPath()); | ||
| logger.error("Unable to delete spill file {}", | ||
| MDC.of(LogKeys.PATH$.MODULE$, spill.file.getPath())); | ||
| } | ||
| } | ||
| } | ||
|
|
@@ -416,8 +419,8 @@ public void insertRecord(Object recordBase, long recordOffset, int length, int p | |
| // for tests | ||
| assert(inMemSorter != null); | ||
| if (inMemSorter.numRecords() >= numElementsForSpillThreshold) { | ||
| logger.info("Spilling data because number of spilledRecords crossed the threshold " + | ||
| numElementsForSpillThreshold); | ||
| logger.info("Spilling data because number of spilledRecords crossed the threshold {}" + | ||
|
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. When re-implementing SPARK-27734, I found that there was something wrong with the logs here.
|
||
| MDC.of(LogKeys.NUM_ELEMENTS_SPILL_THRESHOLD$.MODULE$, numElementsForSpillThreshold)); | ||
| spill(); | ||
| } | ||
|
|
||
|
|
||
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.
It is found that some logic uses
this methodduring migration.