From adb51a9d067a33348d9c54e9abced6cc77256606 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Thu, 9 May 2024 14:30:52 -0700 Subject: [PATCH] Squash of adding initial autotune for number of executors. This also removes a lot of JSON code and drops handling for the classic SparkLens file format instead folks should directly use the history files part2 --- .gitignore | 15 + build.sbt | 2 +- e2e/partioned_table_join.sql | 14 + env_setup.sh | 66 + run_e2e.sh | 72 + scalastyle-config.xml | 117 ++ setup_micro_k8s.sh | 37 + .../qubole/sparklens/QuboleJobListener.scala | 15 +- .../sparklens/QuboleNotebookListener.scala | 15 +- .../sparklens/analyzer/AppAnalyzer.scala | 29 +- .../analyzer/ExecutorTimelineAnalyzer.scala | 58 +- .../analyzer/JobOverlapAnalyzer.scala | 8 +- .../analyzer/StageSkewAnalyzer.scala | 18 +- .../sparklens/app/EventHistoryReporter.scala | 4 +- .../qubole/sparklens/app/ReporterApp.scala | 43 +- .../qubole/sparklens/app/SparklensApp.scala | 12 +- .../qubole/sparklens/common/AppContext.scala | 42 +- .../scala/com/qubole/sparklens/package.scala | 20 - .../scheduler/CompletionEstimator.scala | 4 +- .../scheduler/PQParallelStageScheduler.scala | 4 +- .../sparklens/timespan/ExecutorTimeSpan.scala | 24 +- .../sparklens/timespan/JobTimeSpan.scala | 21 - .../qubole/sparklens/timespan/TimeSpan.scala | 4 + src/test/compatibility-files/version-1.json | 1205 ----------------- src/test/compatibility-files/version-1.output | 14 - src/test/event-history-test-files/local-fresh | 866 ++++++++++++ .../local-fresh-dynamic | 867 ++++++++++++ src/test/scala/CompatibilitySuite.scala | 41 - .../com/qubole/sparklens/TestUtils.scala | 7 +- .../ExecutorTimelineAnalyzerSuite.scala | 77 ++ .../analyzer/JobOverlapAnalyzerSuite.scala | 10 +- .../sparklens/app/EventHistoryTest.scala | 60 + .../sparklens/common/AppContextSuite.scala | 5 +- .../PQParallelStageSchedulerSuite.scala | 9 +- 34 files changed, 2385 insertions(+), 1420 deletions(-) create mode 100644 e2e/partioned_table_join.sql create mode 100755 env_setup.sh create mode 100755 run_e2e.sh create mode 100644 scalastyle-config.xml create mode 100755 setup_micro_k8s.sh delete mode 100644 src/main/scala/com/qubole/sparklens/package.scala delete mode 100644 src/test/compatibility-files/version-1.json delete mode 100644 src/test/compatibility-files/version-1.output create mode 100644 src/test/event-history-test-files/local-fresh create mode 100644 src/test/event-history-test-files/local-fresh-dynamic delete mode 100644 src/test/scala/CompatibilitySuite.scala create mode 100644 src/test/scala/com/qubole/sparklens/analyzer/ExecutorTimelineAnalyzerSuite.scala create mode 100644 src/test/scala/com/qubole/sparklens/app/EventHistoryTest.scala diff --git a/.gitignore b/.gitignore index d9a9908..5cd6ddf 100644 --- a/.gitignore +++ b/.gitignore @@ -64,3 +64,18 @@ distMaven/ # Newt /.newt-cache/ + +# Spark stuff used for e2e testing +*.jar +*.tgz +data/ +spark-*-bin-hadoop*/ +derby.log +warehouse/ +metastore_db + +# metals +project/.bloop +project/metals.sbt +.metals +.bsp \ No newline at end of file diff --git a/build.sbt b/build.sbt index 2e30539..45ab8b1 100644 --- a/build.sbt +++ b/build.sbt @@ -19,7 +19,7 @@ libraryDependencies += "org.apache.httpcomponents" % "httpclient" % "4.5.13" % " libraryDependencies += "org.apache.httpcomponents" % "httpmime" % "4.5.13" % "provided" -libraryDependencies += "org.scalatest" %% "scalatest" % "3.0.9" % "test" +libraryDependencies += "org.scalatest" %% "scalatest" % "3.2.17" % "test" libraryDependencies += "org.junit.platform" % "junit-platform-engine" % "1.6.3" % "test" libraryDependencies += "org.junit.platform" % "junit-platform-launcher" % "1.6.3" % "test" diff --git a/e2e/partioned_table_join.sql b/e2e/partioned_table_join.sql new file mode 100644 index 0000000..1f6dac3 --- /dev/null +++ b/e2e/partioned_table_join.sql @@ -0,0 +1,14 @@ +CREATE TABLE IF NOT EXISTS local.developers ( + username string, + firstname string, + lastname string) +USING iceberg +PARTITIONED BY (username); +CREATE TABLE IF NOT EXISTS local.projects ( + creator string, + projectname string) +USING iceberg +PARTITIONED BY (creator); +INSERT INTO local.developers VALUES("krisnova", "Kris", "Nova"); +INSERT INTO local.projects VALUES("krisnova", "aurae"); +SELECT * FROM local.developers INNER JOIN local.projects ON local.projects.creator = local.developers.username; diff --git a/env_setup.sh b/env_setup.sh new file mode 100755 index 0000000..6b5b29e --- /dev/null +++ b/env_setup.sh @@ -0,0 +1,66 @@ +#!/bin/bash + +set -ex + +# Download Spark and iceberg if not present +SPARK_MAJOR=${SPARK_MAJOR:-"3.3"} +SPARK_VERSION=${SPARK_VERSION:-"${SPARK_MAJOR}.4"} +SCALA_VERSION=${SCALA_VERSION:-"2.12"} +HADOOP_VERSION="3" +SPARK_PATH="$(pwd)/spark-${SPARK_VERSION}-bin-hadoop${HADOOP_VERSION}" +SPARK_FILE="spark-${SPARK_VERSION}-bin-hadoop3.tgz" +ICEBERG_VERSION=${ICEBERG_VERSION:-"1.4.0"} +if [ ! -f "${SPARK_FILE}" ]; then + SPARK_DIST_URL="https://dlcdn.apache.org/spark/spark-${SPARK_VERSION}/${SPARK_FILE}" + SPARK_ARCHIVE_DIST_URL="https://archive.apache.org/dist/spark/spark-${SPARK_VERSION}/${SPARK_FILE}" + if command -v axel &> /dev/null + then + (axel "$SPARK_DIST_URL" || axel "$SPARK_ARCHIVE_DIST_URL") & + else + (wget "$SPARK_DIST_URL" || wget "$SPARK_ARCHIVE_DIST_URL") & + fi +fi +# Download Icberg if not present +ICEBERG_FILE="iceberg-spark-runtime-${SPARK_MAJOR}_${SCALA_VERSION}-${ICEBERG_VERSION}.jar" +if [ ! -f "${ICEBERG_FILE}" ]; then + wget "https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-${SPARK_MAJOR}_${SCALA_VERSION}/${ICEBERG_VERSION}/${ICEBERG_FILE}" -O "${ICEBERG_FILE}" & +fi +# See https://www.mail-archive.com/dev@spark.apache.org/msg27796.html +BC_FILE="bcpkix-jdk15on-1.68.jar" +BC_PROV_FILE="bcprov-jdk15on-1.68.jar" +if [ ! -f "${BC_FILE}" ]; then + wget https://repo1.maven.org/maven2/org/bouncycastle/bcpkix-jdk15on/1.68/bcpkix-jdk15on-1.68.jar +fi +if [ ! -f "${BC_PROV_FILE}" ]; then + wget https://repo1.maven.org/maven2/org/bouncycastle/bcprov-jdk15on/1.68/bcprov-jdk15on-1.68.jar +fi +wait +sleep 1 +# Setup the env +if [ ! -d "${SPARK_PATH}" ]; then + tar -xf "${SPARK_FILE}" +fi + +SPARK_HOME="${SPARK_PATH}" +export SPARK_HOME + +if [ ! -f "${SPARK_PATH}/jars/${ICEBERG_FILE}" ]; then + # Delete the old JAR first. + rm "${SPARK_PATH}/jars/iceberg-spark-runtime*.jar" || echo "No old version to delete." + cp "${ICEBERG_FILE}" "${SPARK_PATH}/jars/${ICEBERG_FILE}" +fi + +# Copy boncy castle for Kube +cp bc*jdk*.jar ${SPARK_PATH}/jars/ + +# Set up for running pyspark and friends +export PATH="${SPARK_PATH}:${SPARK_PATH}/python:${SPARK_PATH}/bin:${SPARK_PATH}/sbin:${PATH}" + +# Make sure we have a history directory +mkdir -p /tmp/spark-events + +mkdir -p ./data/fetched/ +if [ ! -f ./data/fetched/2021 ]; then + wget "https://gender-pay-gap.service.gov.uk/viewing/download-data/2021" -O ./data/fetched/2021 +fi + diff --git a/run_e2e.sh b/run_e2e.sh new file mode 100755 index 0000000..8693f0d --- /dev/null +++ b/run_e2e.sh @@ -0,0 +1,72 @@ +#!/bin/bash + +# Runs an end to end test, updates the output + +set -ex + +rm -rf /tmp/spark-events + +source env_setup.sh + + +sql_file=e2e/partioned_table_join.sql +EXTENSIONS=org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions + +# Run a simple local e2e example + +mkdir -p /tmp/spark-events +${SPARK_HOME}/bin/spark-sql --master local[5] \ + --conf spark.eventLog.enabled=true \ + --conf spark.sql.catalog.spark_catalog=org.apache.iceberg.spark.SparkSessionCatalog \ + --conf spark.sql.catalog.spark_catalog.type=hive \ + --conf spark.sql.catalog.local=org.apache.iceberg.spark.SparkCatalog \ + --conf spark.sql.catalog.local.type=hadoop \ + --conf "spark.sql.catalog.local.warehouse=$PWD/warehouse" \ + --name "fresh" \ + -f "${sql_file}" + + +cp /tmp/spark-events/* ./src/test/event-history-test-files/local-fresh + +# Same example but with dynamic allocation turned on IF AND ONLY IF we have a kube cluster? idk. + +source setup_micro_k8s.sh + +if [ -n "${kube_host}" ]; then + + rm -rf /tmp/spark-events + # This warehouse path sort of works but not really, ideally we'd point to s3/minio but it's enough + # for now. + WAREHOUSE_PATH=${WAREHOUSE_PATH:-/tmp/warehouse} + rm -rf ${WAREHOUSE_PATH} + mkdir -p /tmp/spark-events + ${SPARK_HOME}/bin/spark-sql --master "k8s://${kube_host}" \ + --deploy-mode client \ + --conf spark.eventLog.enabled=true \ + --conf spark.sql.catalog.spark_catalog=org.apache.iceberg.spark.SparkSessionCatalog \ + --conf spark.sql.catalog.spark_catalog.type=hive \ + --conf spark.sql.catalog.local=org.apache.iceberg.spark.SparkCatalog \ + --conf spark.sql.catalog.local.type=hadoop \ + --conf spark.dynamicAllocation.enabled=true \ + --conf "spark.sql.catalog.local.warehouse=/tmp/warehouse" \ + --conf spark.kubernetes.container.image=${image} \ + --conf spark.kubernetes.authenticate.caCertFile=${cert_path} \ + --conf spark.kubernetes.authenticate.submission.caCertFile=${cert_path} \ + --conf spark.dynamicAllocation.shuffleTracking.enabled=true \ + --name "fresh-kube" \ + -f "${sql_file}" + + cp /tmp/spark-events/* ./src/test/event-history-test-files/local-fresh-dynamic + + # TODO: Register trap for this. + cp ~/.kube/${backup_config} ~/.kube/config +else + echo "Not updating dynamically scaled version" +fi + +# Run only the tests that use the "fresh" output +# that is the one where we run Spark as part of the test above +# This allows us to test in CI we support different history file versions provided we're compiled +# for that version. +sbt -DsparkVersion=${SPARK_VERSION} ";clean;compile;testOnly com.qubole.sparklens.app.EventHistoryFileReportingSuite -- -z fresh" + diff --git a/scalastyle-config.xml b/scalastyle-config.xml new file mode 100644 index 0000000..52f8b84 --- /dev/null +++ b/scalastyle-config.xml @@ -0,0 +1,117 @@ + + Scalastyle standard configuration + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/setup_micro_k8s.sh b/setup_micro_k8s.sh new file mode 100755 index 0000000..64b95c2 --- /dev/null +++ b/setup_micro_k8s.sh @@ -0,0 +1,37 @@ +if [command -v microk8s ]; then + echo "microk8s already installed yay!" +elif [ command -v snap ]; then + sudo snap install microk8s --classic + MYUSER=$USER + sudo usermod -a -G microk8s $MYUSER + sudo chown -R $MYUSER ~/.kube + newgrp microk8s +elif [ command -v homebrew ]; then + brew install ubuntu/microk8s/microk8s + microk8s install + echo "Find your cert (see https://www.waitingforcode.com/apache-spark/setting-up-apache-spark-kubernetes-microk8s/read)" + exit 1 +fi +if [ command -v microk8s ]; then + sudo microk8s status --wait-ready + sudo microk8s enable dns + sudo microk8s enable dashboard + sudo microk8s enable storage + sudo microk8s enable registry +fi +kube_host=$(sudo microk8s kubectl cluster-info |grep control |grep http |sed -ne 's/.*\(http[^"]*\).*/\1/p' | cut -d \' -f 2) +if [ -n "${kube_host}" ]; then + cd "${SPARK_HOME}" + kube_host="https://127.0.0.1:16443" + repo=localhost:32000/local-spark + tag=magic + image="${repo}/spark:${tag}" + ./bin/docker-image-tool.sh -r "localhost:32000/local-spark" -t ${tag} build + cert_path=/var/snap/microk8s/current/certs/ca.crt + backup_config="config-$(date +%s)" + cp ~/.kube/config ~/.kube/${backup_config} + # I don't love this but kubeconfig & KUBECONFIG don't seemt to be respected by Spark. + sudo microk8s config > ~/.kube/config + docker push "${image}" + cd - +fi diff --git a/src/main/scala/com/qubole/sparklens/QuboleJobListener.scala b/src/main/scala/com/qubole/sparklens/QuboleJobListener.scala index 5f3670a..2e243e4 100644 --- a/src/main/scala/com/qubole/sparklens/QuboleJobListener.scala +++ b/src/main/scala/com/qubole/sparklens/QuboleJobListener.scala @@ -48,6 +48,7 @@ class QuboleJobListener(sparkConf: SparkConf) extends SparkListener { protected val stageIDToJobID = new mutable.HashMap[Int, Long] protected val failedStages = new ListBuffer[String] protected val appMetrics = new AggregateMetrics() + protected var jobConf: Option[Map[String, String]] = None private def hostCount():Int = hostMap.size @@ -130,6 +131,16 @@ class QuboleJobListener(sparkConf: SparkConf) extends SparkListener { } } + override def onOtherEvent(event: SparkListenerEvent): Unit = { + } + + override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate): Unit = { + // For now we just grab the first Spark conf. + if (jobConf == None) { + jobConf = environmentUpdate.environmentDetails.get("Spark Properties").map(_.toMap) + } + } + override def onApplicationStart(applicationStart: SparkListenerApplicationStart): Unit = { //println(s"Application ${applicationStart.appId} started at ${applicationStart.time}") appInfo.applicationID = applicationStart.appId.getOrElse("NA") @@ -161,7 +172,9 @@ class QuboleJobListener(sparkConf: SparkConf) extends SparkListener { jobMap, jobSQLExecIDMap, stageMap, - stageIDToJobID) + stageIDToJobID, + jobConf + ) EmailReportHelper.generateReport(appContext.toString(), sparkConf) AppAnalyzer.startAnalyzers(appContext) diff --git a/src/main/scala/com/qubole/sparklens/QuboleNotebookListener.scala b/src/main/scala/com/qubole/sparklens/QuboleNotebookListener.scala index b7a2fc4..3b29893 100644 --- a/src/main/scala/com/qubole/sparklens/QuboleNotebookListener.scala +++ b/src/main/scala/com/qubole/sparklens/QuboleNotebookListener.scala @@ -125,13 +125,16 @@ class QuboleNotebookListener(sparkConf: SparkConf) extends QuboleJobListener(spa jobMap, jobSQLExecIDMap, stageMap, - stageIDToJobID) + stageIDToJobID, + jobConf) val out = new mutable.StringBuilder() + val conf: mutable.HashMap[String, String] = new mutable.HashMap() list.foreach(x => { try { - val result = x.analyze(appContext, fromTime, toTime) + val (result, suggestions) = x.analyzeAndSuggest(appContext, fromTime, toTime) + conf ++= suggestions out.append(result) } catch { case e:Throwable => { @@ -140,6 +143,14 @@ class QuboleNotebookListener(sparkConf: SparkConf) extends QuboleJobListener(spa } } }) + if (!conf.isEmpty) { + out.append("Suggested config:\n") + conf.foreach { case (k , v) => + out.append(k + ":" + v + "\n") + } + } else { + out.append("No config changes suggested") + } out.toString() } diff --git a/src/main/scala/com/qubole/sparklens/analyzer/AppAnalyzer.scala b/src/main/scala/com/qubole/sparklens/analyzer/AppAnalyzer.scala index 9112bc7..5427ec9 100644 --- a/src/main/scala/com/qubole/sparklens/analyzer/AppAnalyzer.scala +++ b/src/main/scala/com/qubole/sparklens/analyzer/AppAnalyzer.scala @@ -21,19 +21,33 @@ import java.util.concurrent.TimeUnit import com.qubole.sparklens.common.AppContext +import scala.collection.mutable import scala.collection.mutable.ListBuffer +import org.apache.spark.SparkConf + /* * Interface for creating new Analyzers */ trait AppAnalyzer { + + /** + * Analyze an app and make some string suggestions as well as return a map of config settings + */ + def analyzeAndSuggest(ac: AppContext, startTime: Long, endTime: Long): + (String, Map[String, String]) = { + (analyze(ac), Map.empty[String, String]) + } + def analyze(ac: AppContext): String = { analyze(ac, ac.appInfo.startTime, ac.appInfo.endTime) } def analyze(appContext: AppContext, startTime: Long, endTime: Long): String + + import java.text.SimpleDateFormat val DF = new SimpleDateFormat("hh:mm:ss:SSS") val MINUTES_DF = new SimpleDateFormat("hh:mm") @@ -86,9 +100,14 @@ object AppAnalyzer { list += new StageSkewAnalyzer + val conf: mutable.HashMap[String, String] = new mutable.HashMap() + list.foreach( x => { try { - val output = x.analyze(appContext) + val (output, suggestions) = x.analyzeAndSuggest(appContext, + appContext.appInfo.startTime, + appContext.appInfo.endTime) + conf ++= suggestions println(output) } catch { case e:Throwable => { @@ -97,6 +116,14 @@ object AppAnalyzer { } } }) + if (!conf.isEmpty) { + println("Suggested config:\n") + conf.foreach { case (k , v) => + println(k + ":" + v + "\n") + } + } else { + println("No config changes suggested") + } } } diff --git a/src/main/scala/com/qubole/sparklens/analyzer/ExecutorTimelineAnalyzer.scala b/src/main/scala/com/qubole/sparklens/analyzer/ExecutorTimelineAnalyzer.scala index 56cc40f..6e0edf9 100644 --- a/src/main/scala/com/qubole/sparklens/analyzer/ExecutorTimelineAnalyzer.scala +++ b/src/main/scala/com/qubole/sparklens/analyzer/ExecutorTimelineAnalyzer.scala @@ -1,4 +1,3 @@ - /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -29,8 +28,17 @@ import scala.collection.mutable.ListBuffer class ExecutorTimelineAnalyzer extends AppAnalyzer { def analyze(appContext: AppContext, startTime: Long, endTime: Long): String = { + analyzeAndSuggest(appContext, startTime, endTime)._1 + } + + override def analyzeAndSuggest(appContext: AppContext, startTime: Long, endTime: Long): + (String, Map[String, String]) = { val ac = appContext.filterByStartAndEndTime(startTime, endTime) val out = new mutable.StringBuilder() + val conf = appContext.initialSparkProperties.getOrElse { + out.println("WARNING: No config found using empty config.") + Map.empty[String, String] + } out.println("\nPrinting executors timeline....\n") out.println(s"Total Executors ${ac.executorMap.size}, " + @@ -59,6 +67,52 @@ class ExecutorTimelineAnalyzer extends AppAnalyzer { }) out.println("\nDone printing executors timeline...\n============================\n") - out.toString() + + out.println(s"Config is $conf") + + val isDynamic = conf.getOrElse("spark.dynamicAllocation.enabled", "false").toLowerCase match { + case "true" => true + case _ => false + } + val minExec: Int = conf.getOrElse("spark.dynamicAllocation.minExecutors", "0").toInt + val initialExecs: Int = conf.get("spark.dynamicAllocation.initialExecutors") match { + case None => minExec + case Some(x) => x.toInt + } + // *TODO: Handle resource profiles* + val suggestedParams: Array[(String, String)] = { + // For now we don't recommend turning on dyn alloc here if not already on + // We also don't make any suggestions on partial analysis + if (!isDynamic) { + out.println(s"Not recommedning different initial execs since not dynamic.") + Array.empty[(String, String)] + } else if (appContext.appInfo.startTime != startTime || + appContext.appInfo.endTime != endTime) { + out.println("Not recommending different initial execs since only doing partial analysis.") + Array.empty[(String, String)] + } else { + // A good number of initial execs is probably however main executors were allocated within + // the first 10 minutes of job & did not exit within that same time period. + val magicTime = startTime + 600 + val execTimelines = ac.executorMap.values.toList + val goodInitialExecs = execTimelines.filter { execTimeSpan => + execTimeSpan.getStartTime < magicTime && + (execTimeSpan.getEndTime > magicTime || execTimeSpan.getEndTime > endTime - 600) + }.size + out.println(s"Good initial execs is $goodInitialExecs from $execTimelines") + // "fuzzy logic" aka -- don't change if it's close enough. + if (goodInitialExecs > initialExecs * 1.1 || goodInitialExecs < initialExecs * 0.9) { + Array( + ("spark.dynamicAllocation.initialExecutors", goodInitialExecs.toString) + ) + } else { + out.println("Initial execs within tolerance +-10% leaving alone") + Array.empty[(String, String)] + } + } + } + + + (out.toString(), suggestedParams.toMap) } } diff --git a/src/main/scala/com/qubole/sparklens/analyzer/JobOverlapAnalyzer.scala b/src/main/scala/com/qubole/sparklens/analyzer/JobOverlapAnalyzer.scala index 96c2b27..2039b59 100644 --- a/src/main/scala/com/qubole/sparklens/analyzer/JobOverlapAnalyzer.scala +++ b/src/main/scala/com/qubole/sparklens/analyzer/JobOverlapAnalyzer.scala @@ -1,4 +1,3 @@ - /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -79,8 +78,9 @@ class JobOverlapAnalyzer extends AppAnalyzer { }) if (conflictingJobGroups.size > 0) { - out.println(s"Found ${conflictingJobGroups.size} overlapping JobGroups. Using threadpool for submitting parallel jobs? " + - s"Some calculations might not be reliable.") + out.println(s"Found ${conflictingJobGroups.size} overlapping JobGroups. " + + "These can come from Spark SQL " + + "Some calculations might not be reliable, we will try and improve future versions.") conflictingJobGroups.foreach( x => { out.println(s"Running with overlap: JobGroupID ${x._1} && JobGroupID ${x._2} ") }) @@ -124,5 +124,3 @@ class JobOverlapAnalyzer extends AppAnalyzer { }) } } - - diff --git a/src/main/scala/com/qubole/sparklens/analyzer/StageSkewAnalyzer.scala b/src/main/scala/com/qubole/sparklens/analyzer/StageSkewAnalyzer.scala index fd478c3..cadba0b 100644 --- a/src/main/scala/com/qubole/sparklens/analyzer/StageSkewAnalyzer.scala +++ b/src/main/scala/com/qubole/sparklens/analyzer/StageSkewAnalyzer.scala @@ -1,4 +1,3 @@ - /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -177,14 +176,15 @@ class StageSkewAnalyzer extends AppAnalyzer { } - val totalInput = sts.stageMetrics.map(AggregateMetrics.inputBytesRead).value + sts.stageMetrics.map(AggregateMetrics.shuffleReadBytesRead).value - val totalOutput = sts.stageMetrics.map(AggregateMetrics.outputBytesWritten).value+ sts.stageMetrics.map(AggregateMetrics.shuffleWriteBytesWritten).value - val oiRatio:Float = if (totalInput == 0) { - 0 - }else { - totalOutput.toFloat/totalInput - } - out.println (f"${x}%7s ${stagePercent}%7.2f ${pcm(totalExecutorTime)}%13s ${sts.taskExecutionTimes.length}%7s $parallelismRatio%7.2f $taskSkew%7.2f $taskStageSkew%7.2f $oiRatio%7.2f |* ${writeTimePercent}%6.2f ${readFetchPercent}%13.2f ${gcPercent}%6.2f *|") + val totalInput = sts.stageMetrics.map(AggregateMetrics.inputBytesRead).value + sts.stageMetrics.map(AggregateMetrics.shuffleReadBytesRead).value + val totalOutput = (sts.stageMetrics.map(AggregateMetrics.outputBytesWritten).value + + sts.stageMetrics.map(AggregateMetrics.shuffleWriteBytesWritten).value) + val oiRatio:Float = if (totalInput == 0) { + 0 + } else { + totalOutput.toFloat/totalInput + } + out.println (f"${x}%7s ${stagePercent}%7.2f ${pcm(totalExecutorTime)}%13s ${sts.taskExecutionTimes.length}%7s $parallelismRatio%7.2f $taskSkew%7.2f $taskStageSkew%7.2f $oiRatio%7.2f |* ${writeTimePercent}%6.2f ${readFetchPercent}%13.2f ${gcPercent}%6.2f *|") }) diff --git a/src/main/scala/com/qubole/sparklens/app/EventHistoryReporter.scala b/src/main/scala/com/qubole/sparklens/app/EventHistoryReporter.scala index efa8530..a301948 100644 --- a/src/main/scala/com/qubole/sparklens/app/EventHistoryReporter.scala +++ b/src/main/scala/com/qubole/sparklens/app/EventHistoryReporter.scala @@ -65,6 +65,7 @@ class EventHistoryReporter(file: String, extraConf: List[(String, String)] = Lis } } + // TODO Unify with ReporterApp private def getFilter(eventString: String): Boolean = { implicit val formats = DefaultFormats eventFilter.contains(Json4sWrapper.parse(eventString).extract[Map[String, Any]].get("Event") @@ -81,7 +82,8 @@ class EventHistoryReporter(file: String, extraConf: List[(String, String)] = Lis "SparkListenerJobStart", "SparkListenerJobEnd", "SparkListenerStageSubmitted", - "SparkListenerStageCompleted" + "SparkListenerStageCompleted", + "SparkListenerEnvironmentUpdate", ) } diff --git a/src/main/scala/com/qubole/sparklens/app/ReporterApp.scala b/src/main/scala/com/qubole/sparklens/app/ReporterApp.scala index a8fdb5a..94b30a9 100644 --- a/src/main/scala/com/qubole/sparklens/app/ReporterApp.scala +++ b/src/main/scala/com/qubole/sparklens/app/ReporterApp.scala @@ -36,14 +36,6 @@ object ReporterApp extends App { } } - def startAnalysersFromString(json: String): Unit = { - - implicit val formats = DefaultFormats - val map = Json4sWrapper.parse(json).extract[JValue] - val appContext = AppContext.getContext(map) - startAnalysersFromAppContext(appContext) - } - private def startAnalysersFromAppContext(appContext: AppContext): Unit = { AppAnalyzer.startAnalyzers(appContext) } @@ -51,7 +43,9 @@ object ReporterApp extends App { private def parseInput(): Unit = { getSource match { - case "sparklens" => reportFromSparklensDump(args(0)) + case "sparklens" => { + throw new Exception("Sparklens dumps are no longer supported, please use a history file.") + } case _ => getAppId match { case Some(appId) => // History Provider @@ -70,8 +64,12 @@ object ReporterApp extends App { if ("source".equalsIgnoreCase(splits(0))) { if ("history".equalsIgnoreCase(splits(1))) { return "history" - } else return "sparklens" - } else new IllegalArgumentException(usage) + } else { + return "sparklens" + } + } else { + new IllegalArgumentException(usage) + } } }) return "sparklens" @@ -83,7 +81,9 @@ object ReporterApp extends App { val splits = arg.split("=") if (splits.size == 2 && "appId".equalsIgnoreCase(splits(0))) { return Option(splits(1)) - } else None + } else { + None + } }) None } @@ -98,19 +98,6 @@ object ReporterApp extends App { None } - private def reportFromSparklensDump(file: String): Unit = { - val fs = FileSystem.get(new URI(file), HDFSConfigHelper.getHadoopConf(None)) - - val path = new Path(file) - val byteArray = new Array[Byte](fs.getFileStatus(path).getLen.toInt) - fs.open(path).readFully(byteArray) - - val json = (byteArray.map(_.toChar)).mkString - EmailReportHelper.generateReport(json, conf) - startAnalysersFromString(json) - - } - def reportFromEventHistory(file: String): Unit = { val busKlass = Class.forName("org.apache.spark.scheduler.ReplayListenerBus") val bus = busKlass.newInstance() @@ -129,6 +116,7 @@ object ReporterApp extends App { val replayMethod = busKlass.getMethod("replay", classOf[InputStream], classOf[String], classOf[Boolean], classOf[(String) => Boolean]) + replayMethod.invoke(bus, getDecodedInputStream(file, conf), file, boolean2Boolean(false), getFilter _) } @@ -152,6 +140,7 @@ object ReporterApp extends App { } + // TODO Unify with EventHistoryReporter private def getFilter(eventString: String): Boolean = { implicit val formats = DefaultFormats eventFilter.contains(Json4sWrapper.parse(eventString).extract[Map[String, Any]].get("Event") @@ -168,9 +157,9 @@ object ReporterApp extends App { "SparkListenerJobStart", "SparkListenerJobEnd", "SparkListenerStageSubmitted", - "SparkListenerStageCompleted" + "SparkListenerStageCompleted", + "SparkListenerEnvironmentUpdate" // To get the config ) } } - diff --git a/src/main/scala/com/qubole/sparklens/app/SparklensApp.scala b/src/main/scala/com/qubole/sparklens/app/SparklensApp.scala index 76a4bf6..13722a2 100644 --- a/src/main/scala/com/qubole/sparklens/app/SparklensApp.scala +++ b/src/main/scala/com/qubole/sparklens/app/SparklensApp.scala @@ -22,11 +22,15 @@ class SparklensApp { } } - if (outCapture.toString.nonEmpty) // "yes" + if (outCapture.toString.nonEmpty) { + // "yes" outCapture.toString - else if (errCapture.toString.nonEmpty) // "NO!" + } else if (errCapture.toString.nonEmpty) { + // "NO!" errCapture.toString - else - throw new IllegalArgumentException(s"Failed to analyze event history for appId: $appId and attemptId: $attemptId") + } else { + throw new IllegalArgumentException( + s"Failed to analyze event history for appId: $appId and attemptId: $attemptId") + } } } diff --git a/src/main/scala/com/qubole/sparklens/common/AppContext.scala b/src/main/scala/com/qubole/sparklens/common/AppContext.scala index d5263b8..ad9bd98 100644 --- a/src/main/scala/com/qubole/sparklens/common/AppContext.scala +++ b/src/main/scala/com/qubole/sparklens/common/AppContext.scala @@ -24,14 +24,17 @@ import org.json4s.MappingException import scala.collection.mutable -case class AppContext(appInfo: ApplicationInfo, - appMetrics: AggregateMetrics, - hostMap: mutable.HashMap[String, HostTimeSpan], - executorMap: mutable.HashMap[String, ExecutorTimeSpan], - jobMap: mutable.HashMap[Long, JobTimeSpan], - jobSQLExecIdMap:mutable.HashMap[Long, Long], - stageMap: mutable.HashMap[Int, StageTimeSpan], - stageIDToJobID: mutable.HashMap[Int, Long]) { +import org.apache.spark.SparkConf + +case class AppContext(appInfo: ApplicationInfo, + appMetrics: AggregateMetrics, + hostMap: mutable.HashMap[String, HostTimeSpan], + executorMap: mutable.HashMap[String, ExecutorTimeSpan], + jobMap: mutable.HashMap[Long, JobTimeSpan], + jobSQLExecIdMap: mutable.HashMap[Long, Long], + stageMap: mutable.HashMap[Int, StageTimeSpan], + stageIDToJobID: mutable.HashMap[Int, Long], + initialSparkProperties: Option[Map[String, String]]) { def filterByStartAndEndTime(startTime: Long, endTime: Long): AppContext = { new AppContext(appInfo, @@ -48,7 +51,8 @@ case class AppContext(appInfo: ApplicationInfo, stageMap .filter(x => x._2.startTime >= startTime && x._2.endTime <= endTime), - stageIDToJobID) + stageIDToJobID, + initialSparkProperties) } override def toString(): String = { @@ -61,7 +65,8 @@ case class AppContext(appInfo: ApplicationInfo, "jobMap" -> AppContext.getMap(jobMap), "jobSQLExecIdMap" -> jobSQLExecIdMap, "stageMap" -> AppContext.getMap(stageMap), - "stageIDToJobID" -> stageIDToJobID + "stageIDToJobID" -> stageIDToJobID, + "initialSparkConfig" -> initialSparkProperties ) Serialization.writePretty(map) } @@ -122,22 +127,6 @@ object AppContext { } } - def getContext(json: JValue): AppContext = { - - implicit val formats = DefaultFormats - - new AppContext( - ApplicationInfo.getObject((json \ "appInfo").extract[JValue]), - AggregateMetrics.getAggregateMetrics((json \ "appMetrics").extract[JValue]), - HostTimeSpan.getTimeSpan((json \ "hostMap").extract[Map[String, JValue]]), - ExecutorTimeSpan.getTimeSpan((json \ "executorMap").extract[Map[String, JValue]]), - JobTimeSpan.getTimeSpan((json \ "jobMap").extract[Map[String, JValue]]), - getJobSQLExecIdMap(json, new mutable.HashMap[Long, Long]), - StageTimeSpan.getTimeSpan((json \ "stageMap").extract[Map[String, JValue]]), - getJobToStageMap((json \ "stageIDToJobID").extract[Map[Int, JValue]]) - ) -} - private def getJobToStageMap(json: Map[Int, JValue]): mutable.HashMap[Int, Long] = { implicit val formats = DefaultFormats val map = new mutable.HashMap[Int, Long]() @@ -173,4 +162,3 @@ object AppContext { } } } - diff --git a/src/main/scala/com/qubole/sparklens/package.scala b/src/main/scala/com/qubole/sparklens/package.scala deleted file mode 100644 index 9320524..0000000 --- a/src/main/scala/com/qubole/sparklens/package.scala +++ /dev/null @@ -1,20 +0,0 @@ -package com.qubole - -import org.apache.spark.SparkConf - -package object sparklens { - - private [qubole] def getDumpDirectory(conf: SparkConf): String = { - conf.get("spark.sparklens.data.dir", "/tmp/sparklens/") - } - - private [qubole] def asyncReportingEnabled(conf: SparkConf): Boolean = { - // This will dump info to `getDumpDirectory()` and not run reporting - conf.getBoolean("spark.sparklens.reporting.disabled", false) - } - - private [qubole] def dumpDataEnabled(conf: SparkConf): Boolean = { - /* Even if reporting is in app, we can still dump sparklens data which could be used later */ - conf.getBoolean("spark.sparklens.save.data", true) - } -} diff --git a/src/main/scala/com/qubole/sparklens/scheduler/CompletionEstimator.scala b/src/main/scala/com/qubole/sparklens/scheduler/CompletionEstimator.scala index d9f7e1b..f8eb0c2 100644 --- a/src/main/scala/com/qubole/sparklens/scheduler/CompletionEstimator.scala +++ b/src/main/scala/com/qubole/sparklens/scheduler/CompletionEstimator.scala @@ -72,7 +72,7 @@ object CompletionEstimator { private def processStages(maxStageIDs: List[Int], estate: EstimatorState, scheduler: PQParallelStageScheduler): Long = { //In the worst case we need to push all stages to completion - val MAX_COMPLETION_TRIES = estate.stagesData.size+1 + val MAX_COMPLETION_TRIES = estate.stagesData.size + 1 var completionRetries = 0 while (!maxStageIDs .map(scheduler.isStageComplete(_)) @@ -245,5 +245,3 @@ object CompletionEstimator { }).sum + driverTimeJobBased } } - - diff --git a/src/main/scala/com/qubole/sparklens/scheduler/PQParallelStageScheduler.scala b/src/main/scala/com/qubole/sparklens/scheduler/PQParallelStageScheduler.scala index 7de937b..13d5147 100644 --- a/src/main/scala/com/qubole/sparklens/scheduler/PQParallelStageScheduler.scala +++ b/src/main/scala/com/qubole/sparklens/scheduler/PQParallelStageScheduler.scala @@ -67,7 +67,7 @@ class PQParallelStageScheduler(totalCores: Int, taskCountMap: mutable.HashMap[In if (taskQueue.size == totalCores) { dequeOne() } - taskQueue.enqueue(new QueuedTask(taskTime, wallClock+taskTime, stageID)) + taskQueue.enqueue(new QueuedTask(taskTime, wallClock + taskTime, stageID)) } @@ -101,4 +101,4 @@ class PQParallelStageScheduler(totalCores: Int, taskCountMap: mutable.HashMap[In wallClock } } -} \ No newline at end of file +} diff --git a/src/main/scala/com/qubole/sparklens/timespan/ExecutorTimeSpan.scala b/src/main/scala/com/qubole/sparklens/timespan/ExecutorTimeSpan.scala index 00aa5be..9cb1179 100644 --- a/src/main/scala/com/qubole/sparklens/timespan/ExecutorTimeSpan.scala +++ b/src/main/scala/com/qubole/sparklens/timespan/ExecutorTimeSpan.scala @@ -40,26 +40,8 @@ class ExecutorTimeSpan(val executorID: String, Map("executorID" -> executorID, "hostID" -> hostID, "cores" -> cores, "executorMetrics" -> executorMetrics.getMap()) ++ super.getStartEndTime() } -} - -object ExecutorTimeSpan { - def getTimeSpan(json: Map[String, JValue]): mutable.HashMap[String, ExecutorTimeSpan] = { - implicit val formats = DefaultFormats - val map = new mutable.HashMap[String, ExecutorTimeSpan] - - json.keys.map(key => { - val value = json.get(key).get - val timeSpan = new ExecutorTimeSpan( - (value \ "executorID").extract[String], - (value \ "hostID").extract[String], - (value \ "cores").extract[Int] - ) - timeSpan.executorMetrics = AggregateMetrics.getAggregateMetrics((value - \ "executorMetrics").extract[JValue]) - timeSpan.addStartEnd(value) - map.put(key, timeSpan) - }) - map + override def toString() = { + getMap().toString() } -} \ No newline at end of file +} diff --git a/src/main/scala/com/qubole/sparklens/timespan/JobTimeSpan.scala b/src/main/scala/com/qubole/sparklens/timespan/JobTimeSpan.scala index 1bb27c2..de91a6d 100644 --- a/src/main/scala/com/qubole/sparklens/timespan/JobTimeSpan.scala +++ b/src/main/scala/com/qubole/sparklens/timespan/JobTimeSpan.scala @@ -91,24 +91,3 @@ class JobTimeSpan(val jobID: Long) extends TimeSpan { "stageMap" -> AppContext.getMap(stageMap)) ++ super.getStartEndTime() } } - -object JobTimeSpan { - def getTimeSpan(json: Map[String, JValue]): mutable.HashMap[Long, JobTimeSpan] = { - implicit val formats = DefaultFormats - val map = new mutable.HashMap[Long, JobTimeSpan] - - json.keys.map(key => { - val value = json.get(key).get.extract[JValue] - val timeSpan = new JobTimeSpan((value \ "jobID").extract[Long]) - - timeSpan.jobMetrics = AggregateMetrics.getAggregateMetrics((value \ "jobMetrics") - .extract[JValue]) - timeSpan.stageMap = StageTimeSpan.getTimeSpan((value \ "stageMap").extract[ - immutable.Map[String, JValue]]) - timeSpan.addStartEnd(value) - map.put(key.toLong, timeSpan) - - }) - map - } -} diff --git a/src/main/scala/com/qubole/sparklens/timespan/TimeSpan.scala b/src/main/scala/com/qubole/sparklens/timespan/TimeSpan.scala index 29db3db..6131f84 100644 --- a/src/main/scala/com/qubole/sparklens/timespan/TimeSpan.scala +++ b/src/main/scala/com/qubole/sparklens/timespan/TimeSpan.scala @@ -44,6 +44,10 @@ trait TimeSpan { } def getMap(): Map[String, _ <: Any] + def getStartTime: Long = startTime + + def getEndTime: Long = endTime + def getStartEndTime(): Map[String, Long] = { Map("startTime" -> startTime, "endTime" -> endTime) } diff --git a/src/test/compatibility-files/version-1.json b/src/test/compatibility-files/version-1.json deleted file mode 100644 index d7c94cd..0000000 --- a/src/test/compatibility-files/version-1.json +++ /dev/null @@ -1,1205 +0,0 @@ -{ - "stageIDToJobID": { - "1": 1, - "0": 0 - }, - "appMetrics": { - "count": 8, - "map": { - "taskDuration": { - "value": 1005, - "min": 5, - "max": 264, - "mean": 125.625, - "variance": 115417.87500000001 - }, - "executorRuntime": { - "value": 132, - "min": 3, - "max": 30, - "mean": 16.5, - "variance": 1458.0 - }, - "resultSize": { - "value": 5608, - "min": 699, - "max": 703, - "mean": 701.0, - "variance": 32.0 - }, - "shuffleWriteRecordsWritten": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadRecordsRead": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "memoryBytesSpilled": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadBytesRead": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "outputBytesWritten": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "peakExecutionMemory": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadRemoteBlocks": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleWriteBytesWritten": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "jvmGCTime": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadFetchWaitTime": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "inputBytesRead": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "diskBytesSpilled": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadLocalBlocks": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleWriteTime": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - } - } - }, - "hostMap": { - "localhost": { - "hostID": "localhost", - "hostMetrics": { - "count": 8, - "map": { - "taskDuration": { - "value": 1005, - "min": 5, - "max": 264, - "mean": 125.625, - "variance": 115417.87500000001 - }, - "executorRuntime": { - "value": 132, - "min": 3, - "max": 30, - "mean": 16.5, - "variance": 1458.0 - }, - "resultSize": { - "value": 5608, - "min": 699, - "max": 703, - "mean": 701.0, - "variance": 32.0 - }, - "shuffleWriteRecordsWritten": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadRecordsRead": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "memoryBytesSpilled": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadBytesRead": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "outputBytesWritten": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "peakExecutionMemory": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadRemoteBlocks": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleWriteBytesWritten": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "jvmGCTime": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadFetchWaitTime": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "inputBytesRead": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "diskBytesSpilled": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadLocalBlocks": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleWriteTime": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - } - } - }, - "startTime": 1530689397165, - "endTime": 0 - } - }, - "appInfo": { - "applicationID": "local-1530689397106", - "startTime": 1530689396017, - "endTime": 1530689407138 - }, - "executorMap": { - "driver": { - "startTime": 1530689397165, - "cores": 4, - "executorID": "driver", - "endTime": 0, - "hostID": "localhost", - "executorMetrics": { - "count": 8, - "map": { - "taskDuration": { - "value": 1005, - "min": 5, - "max": 264, - "mean": 125.625, - "variance": 115417.87500000001 - }, - "executorRuntime": { - "value": 132, - "min": 3, - "max": 30, - "mean": 16.5, - "variance": 1458.0 - }, - "resultSize": { - "value": 5608, - "min": 699, - "max": 703, - "mean": 701.0, - "variance": 32.0 - }, - "shuffleWriteRecordsWritten": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadRecordsRead": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "memoryBytesSpilled": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadBytesRead": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "outputBytesWritten": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "peakExecutionMemory": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadRemoteBlocks": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleWriteBytesWritten": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "jvmGCTime": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadFetchWaitTime": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "inputBytesRead": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "diskBytesSpilled": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadLocalBlocks": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleWriteTime": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - } - } - } - } - }, - "stageMap": { - "1": { - "taskPeakMemoryUsage": "[0,0,0,0]", - "minTaskLaunchTime": 1530689404064, - "startTime": 1530689404064, - "taskExecutionTimes": "[3,3,3,3]", - "stageID": 1, - "stageMetrics": { - "count": 4, - "map": { - "taskDuration": { - "value": 23, - "min": 5, - "max": 6, - "mean": 5.75, - "variance": 0.7499999999999998 - }, - "executorRuntime": { - "value": 12, - "min": 3, - "max": 3, - "mean": 3.0, - "variance": 0.0 - }, - "resultSize": { - "value": 2804, - "min": 699, - "max": 703, - "mean": 701.0, - "variance": 16.0 - }, - "shuffleWriteRecordsWritten": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadRecordsRead": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "memoryBytesSpilled": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadBytesRead": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "outputBytesWritten": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "peakExecutionMemory": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadRemoteBlocks": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleWriteBytesWritten": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "jvmGCTime": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadFetchWaitTime": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "inputBytesRead": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "diskBytesSpilled": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadLocalBlocks": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleWriteTime": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - } - } - }, - "endTime": 1530689404070, - "numberOfTasks": 4, - "maxTaskFinishTime": 1530689404070, - "parentStageIDs": "[]" - }, - "0": { - "taskPeakMemoryUsage": "[0,0,0,0]", - "minTaskLaunchTime": 1530689402820, - "startTime": 1530689402820, - "taskExecutionTimes": "[30,30,30,30]", - "stageID": 0, - "stageMetrics": { - "count": 4, - "map": { - "taskDuration": { - "value": 982, - "min": 239, - "max": 264, - "mean": 245.5, - "variance": 456.99999999999983 - }, - "executorRuntime": { - "value": 120, - "min": 30, - "max": 30, - "mean": 30.0, - "variance": 0.0 - }, - "resultSize": { - "value": 2804, - "min": 699, - "max": 703, - "mean": 701.0, - "variance": 16.0 - }, - "shuffleWriteRecordsWritten": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadRecordsRead": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "memoryBytesSpilled": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadBytesRead": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "outputBytesWritten": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "peakExecutionMemory": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadRemoteBlocks": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleWriteBytesWritten": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "jvmGCTime": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadFetchWaitTime": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "inputBytesRead": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "diskBytesSpilled": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadLocalBlocks": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleWriteTime": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - } - } - }, - "endTime": 1530689403087, - "numberOfTasks": 4, - "maxTaskFinishTime": 1530689403087, - "parentStageIDs": "[]" - } - }, - "jobMap": { - "1": { - "startTime": 1530689404058, - "endTime": 1530689404071, - "jobID": 1, - "jobMetrics": { - "count": 4, - "map": { - "taskDuration": { - "value": 23, - "min": 5, - "max": 6, - "mean": 5.75, - "variance": 0.7499999999999998 - }, - "executorRuntime": { - "value": 12, - "min": 3, - "max": 3, - "mean": 3.0, - "variance": 0.0 - }, - "resultSize": { - "value": 2804, - "min": 699, - "max": 703, - "mean": 701.0, - "variance": 16.0 - }, - "shuffleWriteRecordsWritten": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadRecordsRead": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "memoryBytesSpilled": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadBytesRead": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "outputBytesWritten": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "peakExecutionMemory": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadRemoteBlocks": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleWriteBytesWritten": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "jvmGCTime": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadFetchWaitTime": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "inputBytesRead": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "diskBytesSpilled": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadLocalBlocks": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleWriteTime": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - } - } - }, - "stageMap": { - "1": { - "taskPeakMemoryUsage": "[0,0,0,0]", - "minTaskLaunchTime": 1530689404064, - "startTime": 1530689404064, - "taskExecutionTimes": "[3,3,3,3]", - "stageID": 1, - "stageMetrics": { - "count": 4, - "map": { - "taskDuration": { - "value": 23, - "min": 5, - "max": 6, - "mean": 5.75, - "variance": 0.7499999999999998 - }, - "executorRuntime": { - "value": 12, - "min": 3, - "max": 3, - "mean": 3.0, - "variance": 0.0 - }, - "resultSize": { - "value": 2804, - "min": 699, - "max": 703, - "mean": 701.0, - "variance": 16.0 - }, - "shuffleWriteRecordsWritten": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadRecordsRead": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "memoryBytesSpilled": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadBytesRead": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "outputBytesWritten": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "peakExecutionMemory": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadRemoteBlocks": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleWriteBytesWritten": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "jvmGCTime": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadFetchWaitTime": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "inputBytesRead": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "diskBytesSpilled": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadLocalBlocks": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleWriteTime": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - } - } - }, - "endTime": 1530689404070, - "numberOfTasks": 4, - "maxTaskFinishTime": 1530689404070, - "parentStageIDs": "[]" - } - } - }, - "0": { - "startTime": 1530689402577, - "endTime": 1530689403101, - "jobID": 0, - "jobMetrics": { - "count": 4, - "map": { - "taskDuration": { - "value": 982, - "min": 239, - "max": 264, - "mean": 245.5, - "variance": 456.99999999999983 - }, - "executorRuntime": { - "value": 120, - "min": 30, - "max": 30, - "mean": 30.0, - "variance": 0.0 - }, - "resultSize": { - "value": 2804, - "min": 699, - "max": 703, - "mean": 701.0, - "variance": 16.0 - }, - "shuffleWriteRecordsWritten": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadRecordsRead": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "memoryBytesSpilled": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadBytesRead": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "outputBytesWritten": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "peakExecutionMemory": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadRemoteBlocks": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleWriteBytesWritten": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "jvmGCTime": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadFetchWaitTime": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "inputBytesRead": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "diskBytesSpilled": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadLocalBlocks": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleWriteTime": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - } - } - }, - "stageMap": { - "0": { - "taskPeakMemoryUsage": "[0,0,0,0]", - "minTaskLaunchTime": 1530689402820, - "startTime": 1530689402820, - "taskExecutionTimes": "[30,30,30,30]", - "stageID": 0, - "stageMetrics": { - "count": 4, - "map": { - "taskDuration": { - "value": 982, - "min": 239, - "max": 264, - "mean": 245.5, - "variance": 456.99999999999983 - }, - "executorRuntime": { - "value": 120, - "min": 30, - "max": 30, - "mean": 30.0, - "variance": 0.0 - }, - "resultSize": { - "value": 2804, - "min": 699, - "max": 703, - "mean": 701.0, - "variance": 16.0 - }, - "shuffleWriteRecordsWritten": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadRecordsRead": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "memoryBytesSpilled": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadBytesRead": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "outputBytesWritten": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "peakExecutionMemory": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadRemoteBlocks": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleWriteBytesWritten": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "jvmGCTime": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadFetchWaitTime": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "inputBytesRead": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "diskBytesSpilled": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleReadLocalBlocks": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - }, - "shuffleWriteTime": { - "value": 0, - "min": 0, - "max": 0, - "mean": 0.0, - "variance": 0.0 - } - } - }, - "endTime": 1530689403087, - "numberOfTasks": 4, - "maxTaskFinishTime": 1530689403087, - "parentStageIDs": "[]" - } - } - } - } -} \ No newline at end of file diff --git a/src/test/compatibility-files/version-1.output b/src/test/compatibility-files/version-1.output deleted file mode 100644 index 30f4659..0000000 --- a/src/test/compatibility-files/version-1.output +++ /dev/null @@ -1,14 +0,0 @@ -Host localhost startTime 07:29:57:165 executors count 1 -At 07:29 executors added 1 & removed 0 currently available 1 -[ 0 |||||||||||||||||||||||||||||||||||||||| ] -07:30:03:087 Stage 0 ended : maxTaskTime 30 taskCount 4 - Driver WallClock Time 00m 10s 95.17% - Executor WallClock Time 00m 00s 4.83% -Minimum possible time for the app based on the critical path (with infinite resources) 00m 10s -Minimum possible time for the app with same executors, perfect parallelism and zero skew 00m 10s - Total cores available to the app 4 - Executor OneCoreComputeHours used 00h 00m 6.15% - OneCoreComputeHours wasted 00h 00m 93.85% - OneCoreComputeHours wasted Driver 95.17% - Model Error 4% - Executor count 2 (200%) estimated time 00m 10s and estimated cluster utilization 0.16% diff --git a/src/test/event-history-test-files/local-fresh b/src/test/event-history-test-files/local-fresh new file mode 100644 index 0000000..19c376d --- /dev/null +++ b/src/test/event-history-test-files/local-fresh @@ -0,0 +1,866 @@ +{"Event":"SparkListenerLogStart","Spark Version":"3.3.4"} +{"Event":"SparkListenerResourceProfileAdded","Resource Profile Id":0,"Executor Resource Requests":{"cores":{"Resource Name":"cores","Amount":1,"Discovery Script":"","Vendor":""},"memory":{"Resource Name":"memory","Amount":1024,"Discovery Script":"","Vendor":""},"offHeap":{"Resource Name":"offHeap","Amount":0,"Discovery Script":"","Vendor":""}},"Task Resource Requests":{"cpus":{"Resource Name":"cpus","Amount":1.0}}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1714515292284,"Executor ID":"driver","Executor Info":{"Host":"10.69.200.180","Total Cores":5,"Log Urls":{},"Attributes":{},"Resources":{},"Resource Profile Id":0}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"10.69.200.180","Port":41679},"Maximum Memory":455501414,"Timestamp":1714515292293,"Maximum Onheap Memory":455501414,"Maximum Offheap Memory":0} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/lib/jvm/java-17-openjdk-amd64","Java Version":"17.0.10 (Private Build)","Scala Version":"version 2.12.15"},"Spark Properties":{"spark.executor.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.driver.host":"10.69.200.180","spark.eventLog.enabled":"true","spark.driver.port":"37317","spark.sql.catalog.local.type":"hadoop","spark.jars":"","spark.sql.catalog.spark_catalog":"org.apache.iceberg.spark.SparkSessionCatalog","spark.sql.datetime.java8API.enabled":"true","spark.sql.hive.version":"2.3.9","spark.app.name":"fresh","spark.scheduler.mode":"FIFO","spark.submit.pyFiles":"","spark.app.submitTime":"1714515291527","spark.app.startTime":"1714515291883","spark.executor.id":"driver","spark.sql.catalog.local.warehouse":"/home/holden/repos/sparklens/warehouse","spark.driver.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.sql.catalog.local":"org.apache.iceberg.spark.SparkCatalog","spark.submit.deployMode":"client","spark.master":"local[5]","spark.sql.catalogImplementation":"hive","spark.sql.catalog.spark_catalog.type":"hive","spark.app.id":"local-1714515292260"},"Hadoop Properties":{"hadoop.service.shutdown.timeout":"30s","yarn.resourcemanager.amlauncher.thread-count":"50","yarn.sharedcache.enabled":"false","fs.s3a.connection.maximum":"96","yarn.nodemanager.numa-awareness.numactl.cmd":"/usr/bin/numactl","fs.viewfs.overload.scheme.target.o3fs.impl":"org.apache.hadoop.fs.ozone.OzoneFileSystem","fs.s3a.impl":"org.apache.hadoop.fs.s3a.S3AFileSystem","yarn.app.mapreduce.am.scheduler.heartbeat.interval-ms":"1000","yarn.timeline-service.timeline-client.number-of-async-entities-to-merge":"10","hadoop.security.kms.client.timeout":"60","hadoop.http.authentication.kerberos.principal":"HTTP/_HOST@LOCALHOST","mapreduce.jobhistory.loadedjob.tasks.max":"-1","yarn.resourcemanager.application-tag-based-placement.enable":"false","mapreduce.framework.name":"local","yarn.sharedcache.uploader.server.thread-count":"50","yarn.nodemanager.log-aggregation.roll-monitoring-interval-seconds.min":"3600","yarn.nodemanager.linux-container-executor.nonsecure-mode.user-pattern":"^[_.A-Za-z0-9][-@_.A-Za-z0-9]{0,255}?[$]?$","tfile.fs.output.buffer.size":"262144","yarn.app.mapreduce.am.job.task.listener.thread-count":"30","yarn.nodemanager.node-attributes.resync-interval-ms":"120000","yarn.nodemanager.container-log-monitor.interval-ms":"60000","hadoop.security.groups.cache.background.reload.threads":"3","yarn.resourcemanager.webapp.cross-origin.enabled":"false","fs.AbstractFileSystem.ftp.impl":"org.apache.hadoop.fs.ftp.FtpFs","fs.viewfs.overload.scheme.target.gs.impl":"com.google.cloud.hadoop.fs.gcs.GoogleHadoopFS","hadoop.registry.secure":"false","hadoop.shell.safely.delete.limit.num.files":"100","mapreduce.job.acl-view-job":" ","fs.s3a.s3guard.ddb.background.sleep":"25ms","fs.s3a.retry.limit":"7","mapreduce.jobhistory.loadedjobs.cache.size":"5","fs.s3a.s3guard.ddb.table.create":"false","fs.viewfs.overload.scheme.target.s3a.impl":"org.apache.hadoop.fs.s3a.S3AFileSystem","yarn.nodemanager.amrmproxy.enabled":"false","yarn.timeline-service.entity-group-fs-store.with-user-dir":"false","mapreduce.shuffle.pathcache.expire-after-access-minutes":"5","mapreduce.input.fileinputformat.split.minsize":"0","yarn.resourcemanager.container.liveness-monitor.interval-ms":"600000","yarn.resourcemanager.client.thread-count":"50","io.seqfile.compress.blocksize":"1000000","yarn.nodemanager.runtime.linux.docker.allowed-container-runtimes":"runc","fs.viewfs.overload.scheme.target.http.impl":"org.apache.hadoop.fs.http.HttpFileSystem","yarn.resourcemanager.nodemanagers.heartbeat-interval-slowdown-factor":"1.0","yarn.sharedcache.checksum.algo.impl":"org.apache.hadoop.yarn.sharedcache.ChecksumSHA256Impl","yarn.nodemanager.amrmproxy.interceptor-class.pipeline":"org.apache.hadoop.yarn.server.nodemanager.amrmproxy.DefaultRequestInterceptor","yarn.timeline-service.entity-group-fs-store.leveldb-cache-read-cache-size":"10485760","mapreduce.reduce.shuffle.fetch.retry.interval-ms":"1000","mapreduce.task.profile.maps":"0-2","yarn.scheduler.include-port-in-node-name":"false","yarn.nodemanager.admin-env":"MALLOC_ARENA_MAX=$MALLOC_ARENA_MAX","yarn.resourcemanager.node-removal-untracked.timeout-ms":"60000","mapreduce.am.max-attempts":"2","hadoop.security.kms.client.failover.sleep.base.millis":"100","mapreduce.jobhistory.webapp.https.address":"0.0.0.0:19890","yarn.node-labels.fs-store.impl.class":"org.apache.hadoop.yarn.nodelabels.FileSystemNodeLabelsStore","yarn.nodemanager.collector-service.address":"${yarn.nodemanager.hostname}:8048","fs.trash.checkpoint.interval":"0","mapreduce.job.map.output.collector.class":"org.apache.hadoop.mapred.MapTask$MapOutputBuffer","yarn.resourcemanager.node-ip-cache.expiry-interval-secs":"-1","hadoop.http.authentication.signature.secret.file":"*********(redacted)","hadoop.jetty.logs.serve.aliases":"true","yarn.resourcemanager.placement-constraints.handler":"disabled","yarn.timeline-service.handler-thread-count":"10","yarn.resourcemanager.max-completed-applications":"1000","yarn.nodemanager.aux-services.manifest.enabled":"false","yarn.resourcemanager.system-metrics-publisher.enabled":"false","yarn.resourcemanager.placement-constraints.algorithm.class":"org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.algorithm.DefaultPlacementAlgorithm","yarn.sharedcache.webapp.address":"0.0.0.0:8788","fs.s3a.select.input.csv.quote.escape.character":"\\\\","yarn.resourcemanager.delegation.token.renew-interval":"*********(redacted)","yarn.sharedcache.nm.uploader.replication.factor":"10","hadoop.security.groups.negative-cache.secs":"30","yarn.app.mapreduce.task.container.log.backups":"0","mapreduce.reduce.skip.proc-count.auto-incr":"true","fs.viewfs.overload.scheme.target.swift.impl":"org.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem","hadoop.security.group.mapping.ldap.posix.attr.gid.name":"gidNumber","ipc.client.fallback-to-simple-auth-allowed":"false","yarn.nodemanager.resource.memory.enforced":"true","yarn.resourcemanager.system-metrics-publisher.timeline-server-v1.enable-batch":"false","yarn.client.failover-proxy-provider":"org.apache.hadoop.yarn.client.ConfiguredRMFailoverProxyProvider","yarn.timeline-service.http-authentication.simple.anonymous.allowed":"true","ha.health-monitor.check-interval.ms":"1000","yarn.nodemanager.runtime.linux.runc.host-pid-namespace.allowed":"false","hadoop.metrics.jvm.use-thread-mxbean":"false","ipc.[port_number].faircallqueue.multiplexer.weights":"8,4,2,1","yarn.acl.reservation-enable":"false","yarn.resourcemanager.store.class":"org.apache.hadoop.yarn.server.resourcemanager.recovery.FileSystemRMStateStore","yarn.app.mapreduce.am.hard-kill-timeout-ms":"10000","fs.s3a.etag.checksum.enabled":"false","yarn.nodemanager.container-metrics.enable":"true","ha.health-monitor.rpc.connect.max.retries":"1","yarn.timeline-service.client.fd-clean-interval-secs":"60","yarn.resourcemanager.nodemanagers.heartbeat-interval-scaling-enable":"false","yarn.resourcemanager.nodemanagers.heartbeat-interval-ms":"1000","hadoop.common.configuration.version":"3.0.0","fs.s3a.s3guard.ddb.table.capacity.read":"0","yarn.nodemanager.remote-app-log-dir-suffix":"logs","yarn.nodemanager.container-log-monitor.dir-size-limit-bytes":"1000000000","yarn.nodemanager.windows-container.cpu-limit.enabled":"false","yarn.nodemanager.runtime.linux.docker.privileged-containers.allowed":"false","file.blocksize":"67108864","hadoop.http.idle_timeout.ms":"60000","hadoop.registry.zk.retry.ceiling.ms":"60000","yarn.scheduler.configuration.leveldb-store.path":"${hadoop.tmp.dir}/yarn/system/confstore","yarn.sharedcache.store.in-memory.initial-delay-mins":"10","mapreduce.jobhistory.principal":"jhs/_HOST@REALM.TLD","mapreduce.map.skip.proc-count.auto-incr":"true","fs.s3a.committer.name":"file","mapreduce.task.profile.reduces":"0-2","hadoop.zk.num-retries":"1000","yarn.webapp.xfs-filter.enabled":"true","fs.viewfs.overload.scheme.target.hdfs.impl":"org.apache.hadoop.hdfs.DistributedFileSystem","seq.io.sort.mb":"100","yarn.scheduler.configuration.max.version":"100","yarn.timeline-service.webapp.https.address":"${yarn.timeline-service.hostname}:8190","yarn.resourcemanager.scheduler.address":"${yarn.resourcemanager.hostname}:8030","yarn.node-labels.enabled":"false","yarn.resourcemanager.webapp.ui-actions.enabled":"true","mapreduce.task.timeout":"600000","yarn.sharedcache.client-server.thread-count":"50","hadoop.security.groups.shell.command.timeout":"0s","hadoop.security.crypto.cipher.suite":"AES/CTR/NoPadding","yarn.nodemanager.elastic-memory-control.oom-handler":"org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.DefaultOOMHandler","yarn.resourcemanager.connect.max-wait.ms":"900000","fs.defaultFS":"file:///","yarn.minicluster.use-rpc":"false","ipc.[port_number].decay-scheduler.decay-factor":"0.5","fs.har.impl.disable.cache":"true","yarn.webapp.ui2.enable":"false","io.compression.codec.bzip2.library":"system-native","yarn.webapp.filter-invalid-xml-chars":"false","yarn.nodemanager.runtime.linux.runc.layer-mounts-interval-secs":"600","fs.s3a.select.input.csv.record.delimiter":"\\n","fs.s3a.change.detection.source":"etag","ipc.[port_number].backoff.enable":"false","yarn.nodemanager.distributed-scheduling.enabled":"false","mapreduce.shuffle.connection-keep-alive.timeout":"5","yarn.resourcemanager.webapp.https.address":"${yarn.resourcemanager.hostname}:8090","yarn.webapp.enable-rest-app-submissions":"true","mapreduce.jobhistory.address":"0.0.0.0:10020","yarn.resourcemanager.nm-tokens.master-key-rolling-interval-secs":"*********(redacted)","yarn.is.minicluster":"false","yarn.nodemanager.address":"${yarn.nodemanager.hostname}:0","fs.abfss.impl":"org.apache.hadoop.fs.azurebfs.SecureAzureBlobFileSystem","fs.AbstractFileSystem.s3a.impl":"org.apache.hadoop.fs.s3a.S3A","mapreduce.task.combine.progress.records":"10000","yarn.resourcemanager.epoch.range":"0","yarn.resourcemanager.am.max-attempts":"2","yarn.nodemanager.runtime.linux.runc.image-toplevel-dir":"/runc-root","yarn.nodemanager.linux-container-executor.cgroups.hierarchy":"/hadoop-yarn","fs.AbstractFileSystem.wasbs.impl":"org.apache.hadoop.fs.azure.Wasbs","yarn.timeline-service.entity-group-fs-store.cache-store-class":"org.apache.hadoop.yarn.server.timeline.MemoryTimelineStore","yarn.nodemanager.runtime.linux.runc.allowed-container-networks":"host,none,bridge","fs.ftp.transfer.mode":"BLOCK_TRANSFER_MODE","ipc.server.log.slow.rpc":"false","ipc.server.reuseaddr":"true","fs.ftp.timeout":"0","yarn.resourcemanager.node-labels.provider.fetch-interval-ms":"1800000","yarn.router.webapp.https.address":"0.0.0.0:8091","yarn.nodemanager.webapp.cross-origin.enabled":"false","fs.wasb.impl":"org.apache.hadoop.fs.azure.NativeAzureFileSystem","yarn.resourcemanager.auto-update.containers":"false","yarn.app.mapreduce.am.job.committer.cancel-timeout":"60000","yarn.scheduler.configuration.zk-store.parent-path":"/confstore","yarn.nodemanager.default-container-executor.log-dirs.permissions":"710","yarn.app.attempt.diagnostics.limit.kc":"64","fs.viewfs.overload.scheme.target.swebhdfs.impl":"org.apache.hadoop.hdfs.web.SWebHdfsFileSystem","yarn.client.failover-no-ha-proxy-provider":"org.apache.hadoop.yarn.client.DefaultNoHARMFailoverProxyProvider","fs.s3a.change.detection.mode":"server","ftp.bytes-per-checksum":"512","yarn.nodemanager.resource.memory-mb":"-1","fs.AbstractFileSystem.abfs.impl":"org.apache.hadoop.fs.azurebfs.Abfs","yarn.timeline-service.writer.flush-interval-seconds":"60","fs.s3a.fast.upload.active.blocks":"4","yarn.resourcemanager.submission-preprocessor.enabled":"false","hadoop.security.credential.clear-text-fallback":"true","yarn.nodemanager.collector-service.thread-count":"5","ipc.[port_number].scheduler.impl":"org.apache.hadoop.ipc.DefaultRpcScheduler","fs.azure.secure.mode":"false","mapreduce.jobhistory.joblist.cache.size":"20000","fs.ftp.host":"0.0.0.0","yarn.timeline-service.writer.async.queue.capacity":"100","yarn.resourcemanager.fs.state-store.num-retries":"0","yarn.resourcemanager.nodemanager-connect-retries":"10","yarn.nodemanager.log-aggregation.num-log-files-per-app":"30","hadoop.security.kms.client.encrypted.key.cache.low-watermark":"0.3f","fs.s3a.committer.magic.enabled":"true","yarn.timeline-service.client.max-retries":"30","dfs.ha.fencing.ssh.connect-timeout":"30000","yarn.log-aggregation-enable":"false","yarn.system-metrics-publisher.enabled":"false","mapreduce.reduce.markreset.buffer.percent":"0.0","fs.AbstractFileSystem.viewfs.impl":"org.apache.hadoop.fs.viewfs.ViewFs","yarn.resourcemanager.nodemanagers.heartbeat-interval-speedup-factor":"1.0","mapreduce.task.io.sort.factor":"10","yarn.nodemanager.amrmproxy.client.thread-count":"25","ha.failover-controller.new-active.rpc-timeout.ms":"60000","yarn.nodemanager.container-localizer.java.opts":"-Xmx256m","mapreduce.jobhistory.datestring.cache.size":"200000","mapreduce.job.acl-modify-job":" ","yarn.nodemanager.windows-container.memory-limit.enabled":"false","yarn.timeline-service.webapp.address":"${yarn.timeline-service.hostname}:8188","yarn.app.mapreduce.am.job.committer.commit-window":"10000","yarn.nodemanager.container-manager.thread-count":"20","yarn.minicluster.fixed.ports":"false","hadoop.tags.system":"YARN,HDFS,NAMENODE,DATANODE,REQUIRED,SECURITY,KERBEROS,PERFORMANCE,CLIENT\n ,SERVER,DEBUG,DEPRECATED,COMMON,OPTIONAL","yarn.cluster.max-application-priority":"0","yarn.timeline-service.ttl-enable":"true","mapreduce.jobhistory.recovery.store.fs.uri":"${hadoop.tmp.dir}/mapred/history/recoverystore","hadoop.caller.context.signature.max.size":"40","ipc.[port_number].decay-scheduler.backoff.responsetime.enable":"false","yarn.client.load.resource-types.from-server":"false","ha.zookeeper.session-timeout.ms":"10000","ipc.[port_number].decay-scheduler.metrics.top.user.count":"10","tfile.io.chunk.size":"1048576","fs.s3a.s3guard.ddb.table.capacity.write":"0","yarn.dispatcher.print-events-info.threshold":"5000","mapreduce.job.speculative.slowtaskthreshold":"1.0","io.serializations":"org.apache.hadoop.io.serializer.WritableSerialization, org.apache.hadoop.io.serializer.avro.AvroSpecificSerialization, org.apache.hadoop.io.serializer.avro.AvroReflectSerialization","hadoop.security.kms.client.failover.sleep.max.millis":"2000","hadoop.security.group.mapping.ldap.directory.search.timeout":"10000","yarn.scheduler.configuration.store.max-logs":"1000","yarn.nodemanager.node-attributes.provider.fetch-interval-ms":"600000","fs.swift.impl":"org.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem","yarn.nodemanager.local-cache.max-files-per-directory":"8192","hadoop.http.cross-origin.enabled":"false","hadoop.zk.acl":"world:anyone:rwcda","yarn.nodemanager.runtime.linux.runc.image-tag-to-manifest-plugin.num-manifests-to-cache":"10","mapreduce.map.sort.spill.percent":"0.80","yarn.timeline-service.entity-group-fs-store.scan-interval-seconds":"60","yarn.node-attribute.fs-store.impl.class":"org.apache.hadoop.yarn.server.resourcemanager.nodelabels.FileSystemNodeAttributeStore","fs.s3a.retry.interval":"500ms","yarn.timeline-service.client.best-effort":"false","yarn.resourcemanager.webapp.delegation-token-auth-filter.enabled":"*********(redacted)","hadoop.security.group.mapping.ldap.posix.attr.uid.name":"uidNumber","fs.AbstractFileSystem.swebhdfs.impl":"org.apache.hadoop.fs.SWebHdfs","yarn.nodemanager.elastic-memory-control.timeout-sec":"5","fs.s3a.select.enabled":"true","mapreduce.ifile.readahead":"true","yarn.timeline-service.leveldb-timeline-store.ttl-interval-ms":"300000","yarn.timeline-service.reader.webapp.address":"${yarn.timeline-service.webapp.address}","yarn.resourcemanager.placement-constraints.algorithm.pool-size":"1","yarn.timeline-service.hbase.coprocessor.jar.hdfs.location":"/hbase/coprocessor/hadoop-yarn-server-timelineservice.jar","hadoop.security.kms.client.encrypted.key.cache.num.refill.threads":"2","yarn.resourcemanager.scheduler.class":"org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler","yarn.app.mapreduce.am.command-opts":"-Xmx1024m","fs.s3a.metadatastore.fail.on.write.error":"true","hadoop.http.sni.host.check.enabled":"false","mapreduce.cluster.local.dir":"${hadoop.tmp.dir}/mapred/local","io.mapfile.bloom.error.rate":"0.005","fs.client.resolve.topology.enabled":"false","yarn.nodemanager.runtime.linux.allowed-runtimes":"default","yarn.sharedcache.store.class":"org.apache.hadoop.yarn.server.sharedcachemanager.store.InMemorySCMStore","ha.failover-controller.graceful-fence.rpc-timeout.ms":"5000","ftp.replication":"3","fs.getspaceused.jitterMillis":"60000","hadoop.security.uid.cache.secs":"14400","mapreduce.job.maxtaskfailures.per.tracker":"3","fs.s3a.metadatastore.impl":"org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore","io.skip.checksum.errors":"false","yarn.app.mapreduce.client-am.ipc.max-retries-on-timeouts":"3","yarn.timeline-service.webapp.xfs-filter.xframe-options":"SAMEORIGIN","fs.s3a.connection.timeout":"200000","yarn.app.mapreduce.am.webapp.https.enabled":"false","mapreduce.job.max.split.locations":"15","yarn.resourcemanager.nm-container-queuing.max-queue-length":"15","yarn.resourcemanager.delegation-token.always-cancel":"*********(redacted)","hadoop.registry.zk.session.timeout.ms":"60000","yarn.federation.cache-ttl.secs":"300","mapreduce.jvm.system-properties-to-log":"os.name,os.version,java.home,java.runtime.version,java.vendor,java.version,java.vm.name,java.class.path,java.io.tmpdir,user.dir,user.name","yarn.resourcemanager.opportunistic-container-allocation.nodes-used":"10","yarn.timeline-service.entity-group-fs-store.active-dir":"/tmp/entity-file-history/active","mapreduce.shuffle.transfer.buffer.size":"131072","yarn.timeline-service.client.retry-interval-ms":"1000","yarn.timeline-service.flowname.max-size":"0","yarn.http.policy":"HTTP_ONLY","fs.s3a.socket.send.buffer":"8192","fs.AbstractFileSystem.abfss.impl":"org.apache.hadoop.fs.azurebfs.Abfss","yarn.sharedcache.uploader.server.address":"0.0.0.0:8046","yarn.resourcemanager.delegation-token.max-conf-size-bytes":"*********(redacted)","hadoop.http.authentication.token.validity":"*********(redacted)","mapreduce.shuffle.max.connections":"0","yarn.minicluster.yarn.nodemanager.resource.memory-mb":"4096","mapreduce.job.emit-timeline-data":"false","yarn.nodemanager.resource.system-reserved-memory-mb":"-1","hadoop.kerberos.min.seconds.before.relogin":"60","mapreduce.jobhistory.move.thread-count":"3","yarn.resourcemanager.admin.client.thread-count":"1","yarn.dispatcher.drain-events.timeout":"300000","ipc.[port_number].decay-scheduler.backoff.responsetime.thresholds":"10s,20s,30s,40s","fs.s3a.buffer.dir":"${hadoop.tmp.dir}/s3a","hadoop.ssl.enabled.protocols":"TLSv1.2","mapreduce.jobhistory.admin.address":"0.0.0.0:10033","yarn.log-aggregation-status.time-out.ms":"600000","fs.s3a.accesspoint.required":"false","mapreduce.shuffle.port":"13562","yarn.resourcemanager.max-log-aggregation-diagnostics-in-memory":"10","yarn.nodemanager.health-checker.interval-ms":"600000","yarn.router.clientrm.interceptor-class.pipeline":"org.apache.hadoop.yarn.server.router.clientrm.DefaultClientRequestInterceptor","yarn.resourcemanager.zk-appid-node.split-index":"0","ftp.blocksize":"67108864","yarn.nodemanager.runtime.linux.sandbox-mode.local-dirs.permissions":"read","yarn.router.rmadmin.interceptor-class.pipeline":"org.apache.hadoop.yarn.server.router.rmadmin.DefaultRMAdminRequestInterceptor","yarn.nodemanager.log-container-debug-info.enabled":"true","yarn.resourcemanager.activities-manager.app-activities.max-queue-length":"100","yarn.resourcemanager.application-https.policy":"NONE","yarn.client.max-cached-nodemanagers-proxies":"0","yarn.nodemanager.linux-container-executor.cgroups.delete-delay-ms":"20","yarn.nodemanager.delete.debug-delay-sec":"0","yarn.nodemanager.pmem-check-enabled":"true","yarn.nodemanager.disk-health-checker.max-disk-utilization-per-disk-percentage":"90.0","mapreduce.app-submission.cross-platform":"false","yarn.resourcemanager.work-preserving-recovery.scheduling-wait-ms":"10000","yarn.nodemanager.container-retry-minimum-interval-ms":"1000","hadoop.security.groups.cache.secs":"300","yarn.federation.enabled":"false","yarn.workflow-id.tag-prefix":"workflowid:","fs.azure.local.sas.key.mode":"false","ipc.maximum.data.length":"134217728","fs.s3a.endpoint":"s3.amazonaws.com","mapreduce.shuffle.max.threads":"0","yarn.router.pipeline.cache-max-size":"25","yarn.resourcemanager.nm-container-queuing.load-comparator":"QUEUE_LENGTH","yarn.resourcemanager.resource-tracker.nm.ip-hostname-check":"false","hadoop.security.authorization":"false","mapreduce.job.complete.cancel.delegation.tokens":"*********(redacted)","fs.s3a.paging.maximum":"5000","nfs.exports.allowed.hosts":"* rw","yarn.nodemanager.amrmproxy.ha.enable":"false","fs.AbstractFileSystem.gs.impl":"com.google.cloud.hadoop.fs.gcs.GoogleHadoopFS","mapreduce.jobhistory.http.policy":"HTTP_ONLY","yarn.sharedcache.store.in-memory.check-period-mins":"720","hadoop.security.group.mapping.ldap.ssl":"false","fs.s3a.downgrade.syncable.exceptions":"true","yarn.client.application-client-protocol.poll-interval-ms":"200","yarn.scheduler.configuration.leveldb-store.compaction-interval-secs":"86400","yarn.timeline-service.writer.class":"org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineWriterImpl","ha.zookeeper.parent-znode":"/hadoop-ha","yarn.resourcemanager.submission-preprocessor.file-refresh-interval-ms":"60000","yarn.nodemanager.log-aggregation.policy.class":"org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.AllContainerLogAggregationPolicy","mapreduce.reduce.shuffle.merge.percent":"0.66","hadoop.security.group.mapping.ldap.search.filter.group":"(objectClass=group)","yarn.resourcemanager.placement-constraints.scheduler.pool-size":"1","yarn.resourcemanager.activities-manager.cleanup-interval-ms":"5000","yarn.nodemanager.resourcemanager.minimum.version":"NONE","mapreduce.job.speculative.speculative-cap-running-tasks":"0.1","yarn.admin.acl":"*","ipc.[port_number].identity-provider.impl":"org.apache.hadoop.ipc.UserIdentityProvider","yarn.nodemanager.recovery.supervised":"false","yarn.sharedcache.admin.thread-count":"1","yarn.resourcemanager.ha.automatic-failover.enabled":"true","yarn.nodemanager.container-log-monitor.total-size-limit-bytes":"10000000000","mapreduce.reduce.skip.maxgroups":"0","mapreduce.reduce.shuffle.connect.timeout":"180000","yarn.nodemanager.health-checker.scripts":"script","yarn.resourcemanager.address":"${yarn.resourcemanager.hostname}:8032","ipc.client.ping":"true","mapreduce.task.local-fs.write-limit.bytes":"-1","fs.adl.oauth2.access.token.provider.type":"*********(redacted)","mapreduce.shuffle.ssl.file.buffer.size":"65536","yarn.resourcemanager.ha.automatic-failover.embedded":"true","yarn.nodemanager.resource-plugins.gpu.docker-plugin":"nvidia-docker-v1","fs.s3a.s3guard.consistency.retry.interval":"2s","fs.s3a.multipart.purge":"false","yarn.scheduler.configuration.store.class":"file","yarn.resourcemanager.nm-container-queuing.queue-limit-stdev":"1.0f","mapreduce.job.end-notification.max.attempts":"5","mapreduce.output.fileoutputformat.compress.codec":"org.apache.hadoop.io.compress.DefaultCodec","yarn.nodemanager.container-monitor.procfs-tree.smaps-based-rss.enabled":"false","ipc.client.bind.wildcard.addr":"false","yarn.resourcemanager.webapp.rest-csrf.enabled":"false","ha.health-monitor.connect-retry-interval.ms":"1000","yarn.nodemanager.keytab":"/etc/krb5.keytab","mapreduce.jobhistory.keytab":"/etc/security/keytab/jhs.service.keytab","fs.s3a.threads.max":"64","yarn.nodemanager.runtime.linux.docker.image-update":"false","mapreduce.reduce.shuffle.input.buffer.percent":"0.70","fs.viewfs.overload.scheme.target.abfss.impl":"org.apache.hadoop.fs.azurebfs.SecureAzureBlobFileSystem","yarn.dispatcher.cpu-monitor.samples-per-min":"60","hadoop.security.token.service.use_ip":"*********(redacted)","yarn.nodemanager.runtime.linux.docker.allowed-container-networks":"host,none,bridge","yarn.nodemanager.node-labels.resync-interval-ms":"120000","hadoop.tmp.dir":"/tmp/hadoop-${user.name}","mapreduce.job.maps":"2","mapreduce.jobhistory.webapp.rest-csrf.custom-header":"X-XSRF-Header","mapreduce.job.end-notification.max.retry.interval":"5000","yarn.log-aggregation.retain-check-interval-seconds":"-1","yarn.resourcemanager.resource-tracker.client.thread-count":"50","yarn.nodemanager.containers-launcher.class":"org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainersLauncher","yarn.rm.system-metrics-publisher.emit-container-events":"false","yarn.timeline-service.leveldb-timeline-store.start-time-read-cache-size":"10000","yarn.resourcemanager.ha.automatic-failover.zk-base-path":"/yarn-leader-election","io.seqfile.local.dir":"${hadoop.tmp.dir}/io/local","fs.s3a.s3guard.ddb.throttle.retry.interval":"100ms","fs.AbstractFileSystem.wasb.impl":"org.apache.hadoop.fs.azure.Wasb","mapreduce.client.submit.file.replication":"10","mapreduce.jobhistory.minicluster.fixed.ports":"false","fs.s3a.multipart.threshold":"128M","yarn.resourcemanager.webapp.xfs-filter.xframe-options":"SAMEORIGIN","mapreduce.jobhistory.done-dir":"${yarn.app.mapreduce.am.staging-dir}/history/done","ipc.server.purge.interval":"15","ipc.client.idlethreshold":"4000","yarn.nodemanager.linux-container-executor.cgroups.strict-resource-usage":"false","mapreduce.reduce.input.buffer.percent":"0.0","yarn.nodemanager.runtime.linux.docker.userremapping-gid-threshold":"1","yarn.nodemanager.webapp.rest-csrf.enabled":"false","fs.ftp.host.port":"21","ipc.ping.interval":"60000","yarn.resourcemanager.history-writer.multi-threaded-dispatcher.pool-size":"10","yarn.resourcemanager.admin.address":"${yarn.resourcemanager.hostname}:8033","file.client-write-packet-size":"65536","ipc.client.kill.max":"10","mapreduce.reduce.speculative":"true","hadoop.security.key.default.bitlength":"128","mapreduce.job.reducer.unconditional-preempt.delay.sec":"300","yarn.nodemanager.disk-health-checker.interval-ms":"120000","yarn.nodemanager.log.deletion-threads-count":"4","fs.s3a.committer.abort.pending.uploads":"true","yarn.webapp.filter-entity-list-by-user":"false","yarn.resourcemanager.activities-manager.app-activities.ttl-ms":"600000","ipc.client.connection.maxidletime":"10000","mapreduce.task.io.sort.mb":"100","yarn.nodemanager.localizer.client.thread-count":"5","io.erasurecode.codec.rs.rawcoders":"rs_native,rs_java","io.erasurecode.codec.rs-legacy.rawcoders":"rs-legacy_java","yarn.sharedcache.admin.address":"0.0.0.0:8047","yarn.resourcemanager.placement-constraints.algorithm.iterator":"SERIAL","yarn.nodemanager.localizer.cache.cleanup.interval-ms":"600000","hadoop.security.crypto.codec.classes.aes.ctr.nopadding":"org.apache.hadoop.crypto.OpensslAesCtrCryptoCodec, org.apache.hadoop.crypto.JceAesCtrCryptoCodec","mapreduce.job.cache.limit.max-resources-mb":"0","fs.s3a.connection.ssl.enabled":"true","yarn.nodemanager.process-kill-wait.ms":"5000","mapreduce.job.hdfs-servers":"${fs.defaultFS}","yarn.app.mapreduce.am.webapp.https.client.auth":"false","hadoop.workaround.non.threadsafe.getpwuid":"true","fs.df.interval":"60000","ipc.[port_number].decay-scheduler.thresholds":"13,25,50","fs.s3a.multiobjectdelete.enable":"true","yarn.sharedcache.cleaner.resource-sleep-ms":"0","yarn.nodemanager.disk-health-checker.min-healthy-disks":"0.25","hadoop.shell.missing.defaultFs.warning":"false","io.file.buffer.size":"65536","fs.viewfs.overload.scheme.target.wasb.impl":"org.apache.hadoop.fs.azure.NativeAzureFileSystem","hadoop.security.group.mapping.ldap.search.attr.member":"member","hadoop.security.random.device.file.path":"/dev/urandom","hadoop.security.sensitive-config-keys":"*********(redacted)","fs.s3a.s3guard.ddb.max.retries":"9","fs.viewfs.overload.scheme.target.file.impl":"org.apache.hadoop.fs.LocalFileSystem","hadoop.rpc.socket.factory.class.default":"org.apache.hadoop.net.StandardSocketFactory","yarn.intermediate-data-encryption.enable":"false","yarn.resourcemanager.connect.retry-interval.ms":"30000","yarn.nodemanager.container.stderr.pattern":"{*stderr*,*STDERR*}","yarn.scheduler.minimum-allocation-mb":"1024","yarn.app.mapreduce.am.staging-dir":"/tmp/hadoop-yarn/staging","mapreduce.reduce.shuffle.read.timeout":"180000","hadoop.http.cross-origin.max-age":"1800","io.erasurecode.codec.xor.rawcoders":"xor_native,xor_java","fs.s3a.s3guard.consistency.retry.limit":"7","fs.s3a.connection.establish.timeout":"5000","mapreduce.job.running.map.limit":"0","yarn.minicluster.control-resource-monitoring":"false","hadoop.ssl.require.client.cert":"false","hadoop.kerberos.kinit.command":"kinit","yarn.federation.state-store.class":"org.apache.hadoop.yarn.server.federation.store.impl.MemoryFederationStateStore","mapreduce.reduce.log.level":"INFO","hadoop.security.dns.log-slow-lookups.threshold.ms":"1000","mapreduce.job.ubertask.enable":"false","adl.http.timeout":"-1","yarn.resourcemanager.placement-constraints.retry-attempts":"3","hadoop.caller.context.enabled":"false","hadoop.security.group.mapping.ldap.num.attempts":"3","yarn.nodemanager.vmem-pmem-ratio":"2.1","hadoop.rpc.protection":"authentication","ha.health-monitor.rpc-timeout.ms":"45000","yarn.nodemanager.remote-app-log-dir":"/tmp/logs","hadoop.zk.timeout-ms":"10000","fs.s3a.s3guard.cli.prune.age":"86400000","yarn.nodemanager.resource.pcores-vcores-multiplier":"1.0","yarn.nodemanager.runtime.linux.sandbox-mode":"disabled","yarn.app.mapreduce.am.containerlauncher.threadpool-initial-size":"10","fs.viewfs.overload.scheme.target.webhdfs.impl":"org.apache.hadoop.hdfs.web.WebHdfsFileSystem","fs.s3a.committer.threads":"8","hadoop.zk.retry-interval-ms":"1000","hadoop.security.crypto.buffer.size":"8192","yarn.nodemanager.node-labels.provider.fetch-interval-ms":"600000","mapreduce.jobhistory.recovery.store.leveldb.path":"${hadoop.tmp.dir}/mapred/history/recoverystore","yarn.client.failover-retries-on-socket-timeouts":"0","fs.s3a.ssl.channel.mode":"default_jsse","yarn.nodemanager.resource.memory.enabled":"false","fs.azure.authorization.caching.enable":"true","hadoop.security.instrumentation.requires.admin":"false","yarn.nodemanager.delete.thread-count":"4","mapreduce.job.finish-when-all-reducers-done":"true","hadoop.registry.jaas.context":"Client","yarn.timeline-service.leveldb-timeline-store.path":"${hadoop.tmp.dir}/yarn/timeline","io.map.index.interval":"128","yarn.resourcemanager.nm-container-queuing.max-queue-wait-time-ms":"100","fs.abfs.impl":"org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem","mapreduce.job.counters.max":"120","mapreduce.jobhistory.webapp.rest-csrf.enabled":"false","yarn.timeline-service.store-class":"org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore","mapreduce.jobhistory.move.interval-ms":"180000","fs.s3a.change.detection.version.required":"true","yarn.nodemanager.localizer.fetch.thread-count":"4","yarn.resourcemanager.scheduler.client.thread-count":"50","hadoop.ssl.hostname.verifier":"DEFAULT","yarn.timeline-service.leveldb-state-store.path":"${hadoop.tmp.dir}/yarn/timeline","mapreduce.job.classloader":"false","mapreduce.task.profile.map.params":"${mapreduce.task.profile.params}","ipc.client.connect.timeout":"20000","hadoop.security.auth_to_local.mechanism":"hadoop","yarn.timeline-service.app-collector.linger-period.ms":"60000","yarn.nm.liveness-monitor.expiry-interval-ms":"600000","yarn.resourcemanager.reservation-system.planfollower.time-step":"1000","yarn.resourcemanager.activities-manager.scheduler-activities.ttl-ms":"600000","yarn.nodemanager.runtime.linux.docker.enable-userremapping.allowed":"true","yarn.webapp.api-service.enable":"false","yarn.nodemanager.recovery.enabled":"false","mapreduce.job.end-notification.retry.interval":"1000","fs.du.interval":"600000","fs.ftp.impl":"org.apache.hadoop.fs.ftp.FTPFileSystem","yarn.nodemanager.container.stderr.tail.bytes":"4096","yarn.nodemanager.disk-health-checker.disk-free-space-threshold.enabled":"true","hadoop.security.group.mapping.ldap.read.timeout.ms":"60000","hadoop.security.groups.cache.warn.after.ms":"5000","file.bytes-per-checksum":"512","mapreduce.outputcommitter.factory.scheme.s3a":"org.apache.hadoop.fs.s3a.commit.S3ACommitterFactory","hadoop.security.groups.cache.background.reload":"false","yarn.nodemanager.container-monitor.enabled":"true","yarn.nodemanager.elastic-memory-control.enabled":"false","net.topology.script.number.args":"100","mapreduce.task.merge.progress.records":"10000","yarn.nodemanager.localizer.address":"${yarn.nodemanager.hostname}:8040","yarn.timeline-service.keytab":"/etc/krb5.keytab","mapreduce.reduce.shuffle.fetch.retry.timeout-ms":"30000","yarn.resourcemanager.rm.container-allocation.expiry-interval-ms":"600000","yarn.nodemanager.container-executor.exit-code-file.timeout-ms":"2000","mapreduce.fileoutputcommitter.algorithm.version":"1","yarn.resourcemanager.work-preserving-recovery.enabled":"true","mapreduce.map.skip.maxrecords":"0","yarn.sharedcache.root-dir":"/sharedcache","fs.s3a.retry.throttle.limit":"20","hadoop.http.authentication.type":"simple","fs.viewfs.overload.scheme.target.oss.impl":"org.apache.hadoop.fs.aliyun.oss.AliyunOSSFileSystem","mapreduce.job.cache.limit.max-resources":"0","mapreduce.task.userlog.limit.kb":"0","ipc.[port_number].weighted-cost.handler":"1","yarn.resourcemanager.scheduler.monitor.enable":"false","ipc.client.connect.max.retries":"10","hadoop.registry.zk.retry.times":"5","yarn.nodemanager.resource-monitor.interval-ms":"3000","yarn.nodemanager.resource-plugins.gpu.allowed-gpu-devices":"auto","mapreduce.job.sharedcache.mode":"disabled","yarn.nodemanager.webapp.rest-csrf.custom-header":"X-XSRF-Header","mapreduce.shuffle.listen.queue.size":"128","yarn.scheduler.configuration.mutation.acl-policy.class":"org.apache.hadoop.yarn.server.resourcemanager.scheduler.DefaultConfigurationMutationACLPolicy","mapreduce.map.cpu.vcores":"1","yarn.log-aggregation.file-formats":"TFile","yarn.timeline-service.client.fd-retain-secs":"300","fs.s3a.select.output.csv.field.delimiter":",","yarn.nodemanager.health-checker.timeout-ms":"1200000","hadoop.user.group.static.mapping.overrides":"dr.who=;","fs.azure.sas.expiry.period":"90d","fs.s3a.select.output.csv.record.delimiter":"\\n","mapreduce.jobhistory.recovery.store.class":"org.apache.hadoop.mapreduce.v2.hs.HistoryServerFileSystemStateStoreService","fs.viewfs.overload.scheme.target.https.impl":"org.apache.hadoop.fs.http.HttpsFileSystem","fs.s3a.s3guard.ddb.table.sse.enabled":"false","yarn.resourcemanager.fail-fast":"${yarn.fail-fast}","yarn.resourcemanager.proxy-user-privileges.enabled":"false","yarn.router.webapp.interceptor-class.pipeline":"org.apache.hadoop.yarn.server.router.webapp.DefaultRequestInterceptorREST","yarn.nodemanager.resource.memory.cgroups.soft-limit-percentage":"90.0","mapreduce.job.reducer.preempt.delay.sec":"0","hadoop.util.hash.type":"murmur","yarn.nodemanager.disk-validator":"basic","yarn.app.mapreduce.client.job.max-retries":"3","fs.viewfs.overload.scheme.target.ftp.impl":"org.apache.hadoop.fs.ftp.FTPFileSystem","mapreduce.reduce.shuffle.retry-delay.max.ms":"60000","hadoop.security.group.mapping.ldap.connection.timeout.ms":"60000","mapreduce.task.profile.params":"-agentlib:hprof=cpu=samples,heap=sites,force=n,thread=y,verbose=n,file=%s","yarn.app.mapreduce.shuffle.log.backups":"0","yarn.nodemanager.container-diagnostics-maximum-size":"10000","hadoop.registry.zk.retry.interval.ms":"1000","yarn.nodemanager.linux-container-executor.cgroups.delete-timeout-ms":"1000","fs.AbstractFileSystem.file.impl":"org.apache.hadoop.fs.local.LocalFs","yarn.nodemanager.log-aggregation.roll-monitoring-interval-seconds":"-1","mapreduce.jobhistory.cleaner.interval-ms":"86400000","hadoop.registry.zk.quorum":"localhost:2181","yarn.nodemanager.runtime.linux.runc.allowed-container-runtimes":"runc","mapreduce.output.fileoutputformat.compress":"false","yarn.resourcemanager.am-rm-tokens.master-key-rolling-interval-secs":"*********(redacted)","fs.s3a.assumed.role.session.duration":"30m","hadoop.security.group.mapping.ldap.conversion.rule":"none","hadoop.ssl.server.conf":"ssl-server.xml","fs.s3a.retry.throttle.interval":"100ms","seq.io.sort.factor":"100","fs.viewfs.overload.scheme.target.ofs.impl":"org.apache.hadoop.fs.ozone.RootedOzoneFileSystem","yarn.sharedcache.cleaner.initial-delay-mins":"10","mapreduce.client.completion.pollinterval":"5000","hadoop.ssl.keystores.factory.class":"org.apache.hadoop.security.ssl.FileBasedKeyStoresFactory","yarn.app.mapreduce.am.resource.cpu-vcores":"1","yarn.timeline-service.enabled":"false","yarn.nodemanager.runtime.linux.docker.capabilities":"CHOWN,DAC_OVERRIDE,FSETID,FOWNER,MKNOD,NET_RAW,SETGID,SETUID,SETFCAP,SETPCAP,NET_BIND_SERVICE,SYS_CHROOT,KILL,AUDIT_WRITE","yarn.acl.enable":"false","yarn.timeline-service.entity-group-fs-store.done-dir":"/tmp/entity-file-history/done/","hadoop.security.group.mapping.ldap.num.attempts.before.failover":"3","mapreduce.task.profile":"false","hadoop.prometheus.endpoint.enabled":"false","yarn.resourcemanager.fs.state-store.uri":"${hadoop.tmp.dir}/yarn/system/rmstore","mapreduce.jobhistory.always-scan-user-dir":"false","fs.s3a.metadatastore.metadata.ttl":"15m","yarn.nodemanager.opportunistic-containers-use-pause-for-preemption":"false","yarn.nodemanager.linux-container-executor.nonsecure-mode.local-user":"nobody","yarn.timeline-service.reader.class":"org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineReaderImpl","yarn.resourcemanager.configuration.provider-class":"org.apache.hadoop.yarn.LocalConfigurationProvider","yarn.nodemanager.runtime.linux.docker.userremapping-uid-threshold":"1","yarn.resourcemanager.configuration.file-system-based-store":"/yarn/conf","mapreduce.job.cache.limit.max-single-resource-mb":"0","yarn.nodemanager.runtime.linux.docker.stop.grace-period":"10","yarn.resourcemanager.resource-profiles.source-file":"resource-profiles.json","mapreduce.job.dfs.storage.capacity.kill-limit-exceed":"false","yarn.nodemanager.resource.percentage-physical-cpu-limit":"100","mapreduce.jobhistory.client.thread-count":"10","tfile.fs.input.buffer.size":"262144","mapreduce.client.progressmonitor.pollinterval":"1000","yarn.nodemanager.log-dirs":"${yarn.log.dir}/userlogs","yarn.resourcemanager.opportunistic.max.container-allocation.per.am.heartbeat":"-1","fs.automatic.close":"true","yarn.resourcemanager.delegation-token-renewer.thread-retry-interval":"*********(redacted)","fs.s3a.select.input.csv.quote.character":"\"","yarn.nodemanager.hostname":"0.0.0.0","ipc.[port_number].cost-provider.impl":"org.apache.hadoop.ipc.DefaultCostProvider","yarn.nodemanager.runtime.linux.runc.manifest-to-resources-plugin":"org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.runc.HdfsManifestToResourcesPlugin","yarn.nodemanager.remote-app-log-dir-include-older":"true","yarn.nodemanager.resource.memory.cgroups.swappiness":"0","ftp.stream-buffer-size":"4096","yarn.fail-fast":"false","yarn.nodemanager.runtime.linux.runc.layer-mounts-to-keep":"100","yarn.timeline-service.app-aggregation-interval-secs":"15","hadoop.security.group.mapping.ldap.search.filter.user":"(&(objectClass=user)(sAMAccountName={0}))","ipc.[port_number].weighted-cost.lockshared":"10","yarn.nodemanager.container-localizer.log.level":"INFO","yarn.timeline-service.address":"${yarn.timeline-service.hostname}:10200","mapreduce.job.ubertask.maxmaps":"9","fs.s3a.threads.keepalivetime":"60","mapreduce.jobhistory.webapp.rest-csrf.methods-to-ignore":"GET,OPTIONS,HEAD","mapreduce.task.files.preserve.failedtasks":"false","yarn.app.mapreduce.client.job.retry-interval":"2000","ha.failover-controller.graceful-fence.connection.retries":"1","fs.s3a.select.output.csv.quote.escape.character":"\\\\","yarn.resourcemanager.delegation.token.max-lifetime":"*********(redacted)","hadoop.kerberos.keytab.login.autorenewal.enabled":"false","yarn.timeline-service.client.drain-entities.timeout.ms":"2000","yarn.nodemanager.resource-plugins.fpga.vendor-plugin.class":"org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.fpga.IntelFpgaOpenclPlugin","yarn.resourcemanager.nodemanagers.heartbeat-interval-min-ms":"1000","yarn.timeline-service.entity-group-fs-store.summary-store":"org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore","mapreduce.reduce.cpu.vcores":"1","mapreduce.job.encrypted-intermediate-data.buffer.kb":"128","fs.client.resolve.remote.symlinks":"true","yarn.nodemanager.webapp.https.address":"0.0.0.0:8044","hadoop.http.cross-origin.allowed-origins":"*","mapreduce.job.encrypted-intermediate-data":"false","yarn.nodemanager.disk-health-checker.disk-utilization-threshold.enabled":"true","fs.s3a.executor.capacity":"16","yarn.timeline-service.entity-group-fs-store.retain-seconds":"604800","yarn.resourcemanager.metrics.runtime.buckets":"60,300,1440","yarn.timeline-service.generic-application-history.max-applications":"10000","yarn.nodemanager.local-dirs":"${hadoop.tmp.dir}/nm-local-dir","mapreduce.shuffle.connection-keep-alive.enable":"false","yarn.node-labels.configuration-type":"centralized","fs.s3a.path.style.access":"false","yarn.nodemanager.aux-services.mapreduce_shuffle.class":"org.apache.hadoop.mapred.ShuffleHandler","yarn.sharedcache.store.in-memory.staleness-period-mins":"10080","fs.adl.impl":"org.apache.hadoop.fs.adl.AdlFileSystem","yarn.resourcemanager.application.max-tags":"10","hadoop.domainname.resolver.impl":"org.apache.hadoop.net.DNSDomainNameResolver","yarn.resourcemanager.nodemanager.minimum.version":"NONE","mapreduce.jobhistory.webapp.xfs-filter.xframe-options":"SAMEORIGIN","yarn.app.mapreduce.am.staging-dir.erasurecoding.enabled":"false","net.topology.impl":"org.apache.hadoop.net.NetworkTopology","io.map.index.skip":"0","yarn.timeline-service.reader.webapp.https.address":"${yarn.timeline-service.webapp.https.address}","fs.ftp.data.connection.mode":"ACTIVE_LOCAL_DATA_CONNECTION_MODE","mapreduce.job.local-fs.single-disk-limit.check.kill-limit-exceed":"true","fs.azure.buffer.dir":"${hadoop.tmp.dir}/abfs","yarn.scheduler.maximum-allocation-vcores":"4","hadoop.http.cross-origin.allowed-headers":"X-Requested-With,Content-Type,Accept,Origin","yarn.nodemanager.log-aggregation.compression-type":"none","yarn.timeline-service.version":"1.0f","yarn.ipc.rpc.class":"org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC","mapreduce.reduce.maxattempts":"4","yarn.resourcemanager.system-metrics-publisher.timeline-server-v1.batch-size":"1000","hadoop.security.dns.log-slow-lookups.enabled":"false","mapreduce.job.committer.setup.cleanup.needed":"true","hadoop.security.secure.random.impl":"org.apache.hadoop.crypto.random.OpensslSecureRandom","mapreduce.job.running.reduce.limit":"0","fs.s3a.select.errors.include.sql":"false","fs.s3a.connection.request.timeout":"0","ipc.maximum.response.length":"134217728","yarn.resourcemanager.webapp.rest-csrf.methods-to-ignore":"GET,OPTIONS,HEAD","mapreduce.job.token.tracking.ids.enabled":"*********(redacted)","hadoop.caller.context.max.size":"128","yarn.nodemanager.runtime.linux.docker.host-pid-namespace.allowed":"false","yarn.nodemanager.runtime.linux.docker.delayed-removal.allowed":"false","hadoop.registry.system.acls":"sasl:yarn@, sasl:mapred@, sasl:hdfs@","yarn.nodemanager.recovery.dir":"${hadoop.tmp.dir}/yarn-nm-recovery","fs.s3a.fast.upload.buffer":"disk","mapreduce.jobhistory.intermediate-done-dir":"${yarn.app.mapreduce.am.staging-dir}/history/done_intermediate","yarn.app.mapreduce.shuffle.log.separate":"true","yarn.log-aggregation.debug.filesize":"104857600","fs.s3a.max.total.tasks":"32","fs.s3a.readahead.range":"64K","hadoop.http.authentication.simple.anonymous.allowed":"true","fs.s3a.attempts.maximum":"20","hadoop.registry.zk.connection.timeout.ms":"15000","yarn.resourcemanager.delegation-token-renewer.thread-count":"*********(redacted)","yarn.resourcemanager.delegation-token-renewer.thread-timeout":"*********(redacted)","yarn.timeline-service.leveldb-timeline-store.start-time-write-cache-size":"10000","yarn.nodemanager.aux-services.manifest.reload-ms":"0","yarn.nodemanager.emit-container-events":"true","yarn.resourcemanager.resource-profiles.enabled":"false","yarn.timeline-service.hbase-schema.prefix":"prod.","fs.azure.authorization":"false","mapreduce.map.log.level":"INFO","ha.failover-controller.active-standby-elector.zk.op.retries":"3","yarn.resourcemanager.decommissioning-nodes-watcher.poll-interval-secs":"20","mapreduce.output.fileoutputformat.compress.type":"RECORD","yarn.resourcemanager.leveldb-state-store.path":"${hadoop.tmp.dir}/yarn/system/rmstore","yarn.timeline-service.webapp.rest-csrf.custom-header":"X-XSRF-Header","mapreduce.ifile.readahead.bytes":"4194304","yarn.sharedcache.app-checker.class":"org.apache.hadoop.yarn.server.sharedcachemanager.RemoteAppChecker","yarn.nodemanager.linux-container-executor.nonsecure-mode.limit-users":"true","yarn.nodemanager.resource.detect-hardware-capabilities":"false","mapreduce.cluster.acls.enabled":"false","mapreduce.job.speculative.retry-after-no-speculate":"1000","fs.viewfs.overload.scheme.target.abfs.impl":"org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem","hadoop.security.group.mapping.ldap.search.group.hierarchy.levels":"0","yarn.resourcemanager.fs.state-store.retry-interval-ms":"1000","file.stream-buffer-size":"4096","yarn.resourcemanager.application-timeouts.monitor.interval-ms":"3000","mapreduce.map.output.compress.codec":"org.apache.hadoop.io.compress.DefaultCodec","mapreduce.map.speculative":"true","yarn.nodemanager.runtime.linux.runc.image-tag-to-manifest-plugin.hdfs-hash-file":"/runc-root/image-tag-to-hash","mapreduce.job.speculative.retry-after-speculate":"15000","yarn.nodemanager.linux-container-executor.cgroups.mount":"false","yarn.app.mapreduce.am.container.log.backups":"0","yarn.app.mapreduce.am.log.level":"INFO","yarn.nodemanager.runtime.linux.runc.image-tag-to-manifest-plugin":"org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.runc.ImageTagToManifestPlugin","io.bytes.per.checksum":"512","mapreduce.job.reduce.slowstart.completedmaps":"0.05","yarn.timeline-service.http-authentication.type":"simple","hadoop.security.group.mapping.ldap.search.attr.group.name":"cn","yarn.nodemanager.resource-plugins.fpga.allowed-fpga-devices":"auto","yarn.timeline-service.client.internal-timers-ttl-secs":"420","fs.s3a.select.output.csv.quote.character":"\"","hadoop.http.logs.enabled":"true","fs.s3a.block.size":"32M","yarn.sharedcache.client-server.address":"0.0.0.0:8045","yarn.nodemanager.logaggregation.threadpool-size-max":"100","yarn.resourcemanager.hostname":"0.0.0.0","yarn.resourcemanager.delegation.key.update-interval":"86400000","mapreduce.reduce.shuffle.fetch.retry.enabled":"${yarn.nodemanager.recovery.enabled}","mapreduce.map.memory.mb":"-1","mapreduce.task.skip.start.attempts":"2","fs.AbstractFileSystem.hdfs.impl":"org.apache.hadoop.fs.Hdfs","yarn.nodemanager.disk-health-checker.enable":"true","fs.s3a.select.output.csv.quote.fields":"always","ipc.client.tcpnodelay":"true","ipc.client.rpc-timeout.ms":"0","yarn.nodemanager.webapp.rest-csrf.methods-to-ignore":"GET,OPTIONS,HEAD","yarn.resourcemanager.delegation-token-renewer.thread-retry-max-attempts":"*********(redacted)","ipc.client.low-latency":"false","mapreduce.input.lineinputformat.linespermap":"1","yarn.router.interceptor.user.threadpool-size":"5","ipc.client.connect.max.retries.on.timeouts":"45","yarn.timeline-service.leveldb-timeline-store.read-cache-size":"104857600","fs.AbstractFileSystem.har.impl":"org.apache.hadoop.fs.HarFs","mapreduce.job.split.metainfo.maxsize":"10000000","yarn.am.liveness-monitor.expiry-interval-ms":"600000","yarn.resourcemanager.container-tokens.master-key-rolling-interval-secs":"*********(redacted)","yarn.timeline-service.entity-group-fs-store.app-cache-size":"10","yarn.nodemanager.runtime.linux.runc.hdfs-manifest-to-resources-plugin.stat-cache-timeout-interval-secs":"360","fs.s3a.socket.recv.buffer":"8192","rpc.metrics.timeunit":"MILLISECONDS","yarn.resourcemanager.resource-tracker.address":"${yarn.resourcemanager.hostname}:8031","yarn.nodemanager.node-labels.provider.fetch-timeout-ms":"1200000","mapreduce.job.heap.memory-mb.ratio":"0.8","yarn.resourcemanager.leveldb-state-store.compaction-interval-secs":"3600","yarn.resourcemanager.webapp.rest-csrf.custom-header":"X-XSRF-Header","yarn.nodemanager.pluggable-device-framework.enabled":"false","yarn.scheduler.configuration.fs.path":"file://${hadoop.tmp.dir}/yarn/system/schedconf","mapreduce.client.output.filter":"FAILED","hadoop.http.filter.initializers":"org.apache.hadoop.http.lib.StaticUserWebFilter","mapreduce.reduce.memory.mb":"-1","yarn.timeline-service.hostname":"0.0.0.0","file.replication":"1","yarn.nodemanager.container-metrics.unregister-delay-ms":"10000","yarn.nodemanager.container-metrics.period-ms":"-1","mapreduce.fileoutputcommitter.task.cleanup.enabled":"false","yarn.nodemanager.log.retain-seconds":"10800","yarn.timeline-service.entity-group-fs-store.cleaner-interval-seconds":"3600","ipc.[port_number].callqueue.impl":"java.util.concurrent.LinkedBlockingQueue","yarn.resourcemanager.keytab":"/etc/krb5.keytab","hadoop.security.group.mapping.providers.combined":"true","mapreduce.reduce.merge.inmem.threshold":"1000","yarn.timeline-service.recovery.enabled":"false","fs.azure.saskey.usecontainersaskeyforallaccess":"true","yarn.sharedcache.nm.uploader.thread-count":"20","yarn.resourcemanager.nodemanager-graceful-decommission-timeout-secs":"3600","ipc.[port_number].weighted-cost.lockfree":"1","mapreduce.shuffle.ssl.enabled":"false","yarn.timeline-service.hbase.coprocessor.app-final-value-retention-milliseconds":"259200000","yarn.nodemanager.opportunistic-containers-max-queue-length":"0","yarn.resourcemanager.state-store.max-completed-applications":"${yarn.resourcemanager.max-completed-applications}","mapreduce.job.speculative.minimum-allowed-tasks":"10","fs.s3a.aws.credentials.provider":"\n org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider,\n org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider,\n com.amazonaws.auth.EnvironmentVariableCredentialsProvider,\n org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider\n ","yarn.log-aggregation.retain-seconds":"-1","yarn.nodemanager.disk-health-checker.min-free-space-per-disk-mb":"0","mapreduce.jobhistory.max-age-ms":"604800000","hadoop.http.cross-origin.allowed-methods":"GET,POST,HEAD","yarn.resourcemanager.opportunistic-container-allocation.enabled":"false","mapreduce.jobhistory.webapp.address":"0.0.0.0:19888","hadoop.system.tags":"YARN,HDFS,NAMENODE,DATANODE,REQUIRED,SECURITY,KERBEROS,PERFORMANCE,CLIENT\n ,SERVER,DEBUG,DEPRECATED,COMMON,OPTIONAL","yarn.log-aggregation.file-controller.TFile.class":"org.apache.hadoop.yarn.logaggregation.filecontroller.tfile.LogAggregationTFileController","yarn.client.nodemanager-connect.max-wait-ms":"180000","yarn.resourcemanager.webapp.address":"${yarn.resourcemanager.hostname}:8088","mapreduce.jobhistory.recovery.enable":"false","mapreduce.reduce.shuffle.parallelcopies":"5","fs.AbstractFileSystem.webhdfs.impl":"org.apache.hadoop.fs.WebHdfs","fs.trash.interval":"0","yarn.app.mapreduce.client.max-retries":"3","hadoop.security.authentication":"simple","mapreduce.task.profile.reduce.params":"${mapreduce.task.profile.params}","yarn.app.mapreduce.am.resource.mb":"1536","mapreduce.input.fileinputformat.list-status.num-threads":"1","yarn.nodemanager.container-executor.class":"org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor","io.mapfile.bloom.size":"1048576","yarn.timeline-service.ttl-ms":"604800000","yarn.resourcemanager.nm-container-queuing.min-queue-length":"5","yarn.nodemanager.resource.cpu-vcores":"-1","mapreduce.job.reduces":"1","fs.s3a.multipart.size":"64M","fs.s3a.select.input.csv.comment.marker":"#","yarn.scheduler.minimum-allocation-vcores":"1","mapreduce.job.speculative.speculative-cap-total-tasks":"0.01","hadoop.ssl.client.conf":"ssl-client.xml","mapreduce.job.queuename":"default","mapreduce.job.encrypted-intermediate-data-key-size-bits":"128","fs.s3a.metadatastore.authoritative":"false","ipc.[port_number].weighted-cost.response":"1","yarn.nodemanager.webapp.xfs-filter.xframe-options":"SAMEORIGIN","ha.health-monitor.sleep-after-disconnect.ms":"1000","yarn.app.mapreduce.shuffle.log.limit.kb":"0","hadoop.security.group.mapping":"org.apache.hadoop.security.JniBasedUnixGroupsMappingWithFallback","yarn.client.application-client-protocol.poll-timeout-ms":"-1","mapreduce.jobhistory.jhist.format":"binary","mapreduce.task.stuck.timeout-ms":"600000","yarn.resourcemanager.application.max-tag.length":"100","yarn.resourcemanager.ha.enabled":"false","dfs.client.ignore.namenode.default.kms.uri":"false","hadoop.http.staticuser.user":"dr.who","mapreduce.task.exit.timeout.check-interval-ms":"20000","mapreduce.jobhistory.intermediate-user-done-dir.permissions":"770","mapreduce.task.exit.timeout":"60000","yarn.nodemanager.linux-container-executor.resources-handler.class":"org.apache.hadoop.yarn.server.nodemanager.util.DefaultLCEResourcesHandler","mapreduce.reduce.shuffle.memory.limit.percent":"0.25","yarn.resourcemanager.reservation-system.enable":"false","mapreduce.map.output.compress":"false","ha.zookeeper.acl":"world:anyone:rwcda","ipc.server.max.connections":"0","yarn.nodemanager.runtime.linux.docker.default-container-network":"host","yarn.router.webapp.address":"0.0.0.0:8089","yarn.scheduler.maximum-allocation-mb":"8192","yarn.resourcemanager.scheduler.monitor.policies":"org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy","yarn.sharedcache.cleaner.period-mins":"1440","yarn.nodemanager.resource-plugins.gpu.docker-plugin.nvidia-docker-v1.endpoint":"http://localhost:3476/v1.0/docker/cli","yarn.app.mapreduce.am.container.log.limit.kb":"0","ipc.client.connect.retry.interval":"1000","yarn.timeline-service.http-cross-origin.enabled":"false","fs.wasbs.impl":"org.apache.hadoop.fs.azure.NativeAzureFileSystem$Secure","yarn.resourcemanager.nodemanagers.heartbeat-interval-max-ms":"1000","yarn.federation.subcluster-resolver.class":"org.apache.hadoop.yarn.server.federation.resolver.DefaultSubClusterResolverImpl","yarn.resourcemanager.zk-state-store.parent-path":"/rmstore","fs.s3a.select.input.csv.field.delimiter":",","mapreduce.jobhistory.cleaner.enable":"true","yarn.timeline-service.client.fd-flush-interval-secs":"10","hadoop.security.kms.client.encrypted.key.cache.expiry":"43200000","yarn.client.nodemanager-client-async.thread-pool-max-size":"500","mapreduce.map.maxattempts":"4","yarn.resourcemanager.nm-container-queuing.sorting-nodes-interval-ms":"1000","fs.s3a.committer.staging.tmp.path":"tmp/staging","yarn.nodemanager.sleep-delay-before-sigkill.ms":"250","yarn.resourcemanager.nm-container-queuing.min-queue-wait-time-ms":"10","mapreduce.job.end-notification.retry.attempts":"0","yarn.nodemanager.resource.count-logical-processors-as-cores":"false","hadoop.registry.zk.root":"/registry","adl.feature.ownerandgroup.enableupn":"false","yarn.resourcemanager.zk-max-znode-size.bytes":"1048576","mapreduce.job.reduce.shuffle.consumer.plugin.class":"org.apache.hadoop.mapreduce.task.reduce.Shuffle","yarn.resourcemanager.delayed.delegation-token.removal-interval-ms":"*********(redacted)","yarn.nodemanager.localizer.cache.target-size-mb":"10240","fs.s3a.committer.staging.conflict-mode":"append","mapreduce.client.libjars.wildcard":"true","fs.s3a.committer.staging.unique-filenames":"true","yarn.nodemanager.node-attributes.provider.fetch-timeout-ms":"1200000","fs.s3a.list.version":"2","ftp.client-write-packet-size":"65536","ipc.[port_number].weighted-cost.lockexclusive":"100","fs.AbstractFileSystem.adl.impl":"org.apache.hadoop.fs.adl.Adl","yarn.nodemanager.container-log-monitor.enable":"false","hadoop.security.key.default.cipher":"AES/CTR/NoPadding","yarn.client.failover-retries":"0","fs.s3a.multipart.purge.age":"86400","mapreduce.job.local-fs.single-disk-limit.check.interval-ms":"5000","net.topology.node.switch.mapping.impl":"org.apache.hadoop.net.ScriptBasedMapping","yarn.nodemanager.amrmproxy.address":"0.0.0.0:8049","ipc.server.listen.queue.size":"256","ipc.[port_number].decay-scheduler.period-ms":"5000","yarn.nodemanager.runtime.linux.runc.image-tag-to-manifest-plugin.cache-refresh-interval-secs":"60","map.sort.class":"org.apache.hadoop.util.QuickSort","fs.viewfs.rename.strategy":"SAME_MOUNTPOINT","hadoop.security.kms.client.authentication.retry-count":"1","fs.permissions.umask-mode":"022","fs.s3a.assumed.role.credentials.provider":"org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider","yarn.nodemanager.runtime.linux.runc.privileged-containers.allowed":"false","yarn.nodemanager.vmem-check-enabled":"true","yarn.nodemanager.numa-awareness.enabled":"false","yarn.nodemanager.recovery.compaction-interval-secs":"3600","yarn.app.mapreduce.client-am.ipc.max-retries":"3","yarn.resourcemanager.system-metrics-publisher.timeline-server-v1.interval-seconds":"60","yarn.federation.registry.base-dir":"yarnfederation/","yarn.nodemanager.health-checker.run-before-startup":"false","mapreduce.job.max.map":"-1","mapreduce.job.local-fs.single-disk-limit.bytes":"-1","mapreduce.shuffle.pathcache.concurrency-level":"16","mapreduce.job.ubertask.maxreduces":"1","mapreduce.shuffle.pathcache.max-weight":"10485760","hadoop.security.kms.client.encrypted.key.cache.size":"500","hadoop.security.java.secure.random.algorithm":"SHA1PRNG","ha.failover-controller.cli-check.rpc-timeout.ms":"20000","mapreduce.jobhistory.jobname.limit":"50","fs.s3a.select.input.compression":"none","yarn.client.nodemanager-connect.retry-interval-ms":"10000","ipc.[port_number].scheduler.priority.levels":"4","yarn.timeline-service.state-store-class":"org.apache.hadoop.yarn.server.timeline.recovery.LeveldbTimelineStateStore","yarn.nodemanager.env-whitelist":"JAVA_HOME,HADOOP_COMMON_HOME,HADOOP_HDFS_HOME,HADOOP_CONF_DIR,CLASSPATH_PREPEND_DISTCACHE,HADOOP_YARN_HOME,HADOOP_HOME,PATH,LANG,TZ","yarn.sharedcache.nested-level":"3","yarn.timeline-service.webapp.rest-csrf.methods-to-ignore":"GET,OPTIONS,HEAD","fs.azure.user.agent.prefix":"unknown","yarn.resourcemanager.zk-delegation-token-node.split-index":"*********(redacted)","yarn.nodemanager.numa-awareness.read-topology":"false","yarn.nodemanager.webapp.address":"${yarn.nodemanager.hostname}:8042","rpc.metrics.quantile.enable":"false","yarn.registry.class":"org.apache.hadoop.registry.client.impl.FSRegistryOperationsService","mapreduce.jobhistory.admin.acl":"*","yarn.resourcemanager.system-metrics-publisher.dispatcher.pool-size":"10","yarn.scheduler.queue-placement-rules":"user-group","hadoop.http.authentication.kerberos.keytab":"${user.home}/hadoop.keytab","yarn.resourcemanager.recovery.enabled":"false","fs.s3a.select.input.csv.header":"none","yarn.nodemanager.runtime.linux.runc.hdfs-manifest-to-resources-plugin.stat-cache-size":"500","yarn.timeline-service.webapp.rest-csrf.enabled":"false","yarn.nodemanager.disk-health-checker.min-free-space-per-disk-watermark-high-mb":"0"},"System Properties":{"java.io.tmpdir":"/tmp","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"17","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Private Build","java.vm.specification.version":"17","user.home":"/home/holden","sun.arch.data.model":"64","sun.boot.library.path":"/usr/lib/jvm/java-17-openjdk-amd64/lib","user.dir":"/home/holden/repos/sparklens","java.library.path":"/usr/java/packages/lib:/usr/lib/x86_64-linux-gnu/jni:/lib/x86_64-linux-gnu:/usr/lib/x86_64-linux-gnu:/usr/lib/jni:/lib:/usr/lib","os.arch":"amd64","java.vm.version":"17.0.10+7-Ubuntu-122.04.1","jetty.git.hash":"6b67c5719d1f4371b33655ff2d047d24e171e49a","java.runtime.version":"17.0.10+7-Ubuntu-122.04.1","java.vm.info":"mixed mode, sharing","java.runtime.name":"OpenJDK Runtime Environment","java.version.date":"2024-01-16","file.separator":"/","java.class.version":"61.0","java.specification.name":"Java Platform API Specification","file.encoding":"UTF-8","user.timezone":"America/Los_Angeles","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","java.vm.compressedOopsMode":"32-bit","os.version":"6.5.0-28-generic","native.encoding":"UTF-8","java.vm.specification.vendor":"Oracle Corporation","sun.stderr.encoding":"UTF-8","user.country":"US","sun.jnu.encoding":"UTF-8","user.language":"en","sun.stdout.encoding":"UTF-8","java.vendor.url":"Unknown","os.name":"Linux","java.vm.vendor":"Private Build","jdk.debug":"release","java.vendor.url.bug":"Unknown","user.name":"holden","java.vm.name":"OpenJDK 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master local[5] --conf spark.sql.catalog.spark_catalog.type=hive --conf spark.sql.catalog.spark_catalog=org.apache.iceberg.spark.SparkSessionCatalog --conf spark.sql.catalog.local=org.apache.iceberg.spark.SparkCatalog --conf spark.eventLog.enabled=true --conf spark.sql.catalog.local.warehouse=/home/holden/repos/sparklens/warehouse --conf spark.sql.catalog.local.type=hadoop --class org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver --name fresh spark-internal -f e2e/partioned_table_join.sql","java.home":"/usr/lib/jvm/java-17-openjdk-amd64","java.version":"17.0.10","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/httpclient-4.5.13.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jodd-core-3.5.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/JLargeArrays-1.5.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/scala-library-2.12.15.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/javassist-3.25.0-GA.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/commons-lang3-3.12.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/protobuf-java-2.5.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/compress-lzf-1.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hive-shims-common-2.3.9.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-network-shuffle_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hadoop-client-runtime-3.3.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-discovery-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/commons-io-2.11.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/breeze_2.12-1.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/snappy-java-1.1.8.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jakarta.inject-2.6.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spire-platform_2.12-0.17.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jakarta.annotation-api-1.3.5.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/commons-compress-1.21.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hive-beeline-2.3.9.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/libthrift-0.12.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/metrics-json-4.2.7.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-kvstore_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/netty-transport-native-epoll-4.1.74.Final-linux-aarch_64.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/scala-collection-compat_2.12-2.1.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/netty-handler-4.1.74.Final.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/parquet-encoding-1.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/scala-xml_2.12-1.2.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/log4j-1.2-api-2.17.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-unsafe_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/javolution-5.5.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jakarta.servlet-api-4.0.3.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/netty-resolver-4.1.74.Final.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jakarta.validation-api-2.0.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/datanucleus-rdbms-4.1.19.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hive-common-2.3.9.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/cats-kernel_2.12-2.1.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/opencsv-2.3.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/janino-3.0.16.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/velocity-1.5.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/univocity-parsers-2.9.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jersey-server-2.36.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-events-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-network-common_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/zjsonpatch-0.3.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/dropwizard-metrics-hadoop-metrics2-reporter-0.1.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-admissionregistration-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/json4s-ast_2.12-3.7.0-M11.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/transaction-api-1.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jackson-datatype-jsr310-2.13.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-hive_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hadoop-client-api-3.3.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-mesos_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/json4s-jackson_2.12-3.7.0-M11.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/logging-interceptor-3.12.12.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jackson-databind-2.13.4.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-sketch_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/guava-14.0.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/pickle-1.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-streaming_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-apiextensions-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/metrics-jmx-4.2.7.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-storageclass-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/leveldbjni-all-1.8.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/audience-annotations-0.5.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/commons-crypto-1.1.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/parquet-format-structures-1.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/netty-buffer-4.1.74.Final.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spire-util_2.12-0.17.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/iceberg-spark-runtime-3.3_2.12-1.4.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jline-2.14.6.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/HikariCP-2.5.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/ST4-4.0.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/bonecp-0.8.0.RELEASE.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/lz4-java-1.8.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jersey-container-servlet-2.36.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/arpack-2.2.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/commons-logging-1.1.3.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hive-service-rpc-3.1.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/shims-0.9.25.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/commons-codec-1.15.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-metrics-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jackson-dataformat-yaml-2.13.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/commons-compiler-3.0.16.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/commons-collections-3.2.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/chill_2.12-0.10.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/arrow-vector-7.0.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kryo-shaded-4.0.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/curator-recipes-2.13.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-tags_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jcl-over-slf4j-1.7.32.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/scala-reflect-2.12.15.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/metrics-graphite-4.2.7.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/arrow-memory-netty-7.0.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hive-shims-0.23-2.3.9.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/parquet-hadoop-1.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spire-macros_2.12-0.17.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-tags_2.12-3.3.4-tests.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/bcprov-jdk15on-1.68.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/conf/":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jackson-core-2.13.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/json4s-scalap_2.12-3.7.0-M11.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/json-1.8.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/derby-10.14.2.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-core-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jdo-api-3.0.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/scala-compiler-2.12.15.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-repl_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jaxb-runtime-2.3.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jersey-client-2.36.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/JTransforms-3.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-client-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/breeze-macros_2.12-1.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hadoop-shaded-guava-1.1.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/json4s-core_2.12-3.7.0-M11.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/arrow-format-7.0.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hive-vector-code-gen-2.3.9.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/okhttp-3.12.12.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/xz-1.9.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hive-jdbc-2.3.9.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hive-exec-2.3.9-core.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/slf4j-api-1.7.32.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-scheduling-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/aopalliance-repackaged-2.6.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-coordination-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-rbac-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/datanucleus-core-4.1.17.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/RoaringBitmap-0.9.25.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/core-1.1.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jsr305-3.0.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/parquet-common-1.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/antlr-runtime-3.5.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/commons-math3-3.6.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/aircompressor-0.21.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hadoop-yarn-server-web-proxy-3.3.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/orc-core-1.7.10.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/orc-shims-1.7.10.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/arpack_combined_all-0.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-catalyst_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/generex-1.0.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/netty-transport-native-kqueue-4.1.74.Final-osx-x86_64.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/py4j-0.10.9.5.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jakarta.ws.rs-api-2.1.6.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/commons-text-1.10.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-node-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-extensions-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/netty-all-4.1.74.Final.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/ivy-2.5.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/flatbuffers-java-1.12.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/arrow-memory-core-7.0.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hive-cli-2.3.9.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/netty-tcnative-classes-2.0.48.Final.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/bcpkix-jdk15on-1.68.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/netty-transport-native-epoll-4.1.74.Final-linux-x86_64.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/httpcore-4.4.14.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/log4j-core-2.17.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/commons-dbcp-1.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/netty-transport-classes-epoll-4.1.74.Final.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/osgi-resource-locator-1.0.3.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/curator-client-2.13.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/avro-ipc-1.11.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/stream-2.9.6.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-hive-thriftserver_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/datanucleus-api-jdo-4.2.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/okio-1.14.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/rocksdbjni-6.20.3.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-flowcontrol-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/netty-transport-native-unix-common-4.1.74.Final.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-launcher_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jul-to-slf4j-1.7.32.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-certificates-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/netty-common-4.1.74.Final.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/algebra_2.12-2.0.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-mllib_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jpam-1.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/avro-1.11.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/avro-mapred-1.11.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/orc-mapreduce-1.7.10.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-batch-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/commons-cli-1.5.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-kubernetes_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jackson-module-scala_2.12-2.13.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/scala-parser-combinators_2.12-1.1.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/gson-2.2.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spire_2.12-0.17.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/zookeeper-jute-3.6.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/oro-2.0.8.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hk2-utils-2.6.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-core_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/blas-2.2.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/annotations-17.0.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/log4j-api-2.17.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/tink-1.6.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/chill-java-0.10.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hive-storage-api-2.7.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-networking-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/curator-framework-2.13.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-mllib-local_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/zookeeper-3.6.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/commons-collections4-4.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/minlog-1.3.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hive-serde-2.3.9.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/snakeyaml-1.31.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/lapack-2.2.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/commons-lang-2.6.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jersey-common-2.36.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jersey-hk2-2.36.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/parquet-jackson-1.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-yarn_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/istack-commons-runtime-3.0.8.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-common-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/automaton-1.11-8.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-sql_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/antlr4-runtime-4.8.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/netty-codec-4.1.74.Final.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jta-1.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jakarta.xml.bind-api-2.3.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/metrics-jvm-4.2.7.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/shapeless_2.12-2.3.7.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-apps-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-graphx_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/log4j-slf4j-impl-2.17.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/paranamer-2.8.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/xbean-asm9-shaded-4.20.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jackson-annotations-2.13.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/activation-1.1.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hk2-locator-2.6.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hk2-api-2.6.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-policy-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/netty-transport-native-kqueue-4.1.74.Final-osx-aarch_64.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/netty-transport-4.1.74.Final.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/mesos-1.4.3-shaded-protobuf.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/super-csv-2.2.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/joda-time-2.10.13.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/objenesis-3.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hive-shims-scheduler-2.3.9.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/javax.jdo-3.2.0-m3.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hive-shims-2.3.9.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/stax-api-1.0.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/metrics-core-4.2.7.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/zstd-jni-1.5.2-1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/libfb303-0.9.3.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/netty-transport-classes-kqueue-4.1.74.Final.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/commons-pool-1.5.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-autoscaling-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hive-llap-common-2.3.9.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hive-metastore-2.3.9.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jersey-container-servlet-core-2.36.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/threeten-extra-1.5.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/parquet-column-1.12.2.jar":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"fresh","App ID":"local-1714515292260","Timestamp":1714515291883,"User":"holden"} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":0,"description":"CREATE TABLE IF NOT EXISTS local.developers (\n username string,\n firstname string,\n lastname string)\nUSING iceberg\nPARTITIONED BY (username)","details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nCreateTable (1)\n\n\n(1) CreateTable\nArguments: org.apache.iceberg.spark.SparkCatalog@295e9aef, developers, [StructField(username,StringType,true), StructField(firstname,StringType,true), StructField(lastname,StringType,true)], [identity(username)], TableSpec(Map(),Some(iceberg),Map(),None,None,None,false), true\n\n","sparkPlanInfo":{"nodeName":"CreateTable","simpleString":"CreateTable org.apache.iceberg.spark.SparkCatalog@295e9aef, developers, [StructField(username,StringType,true), StructField(firstname,StringType,true), StructField(lastname,StringType,true)], [identity(username)], TableSpec(Map(),Some(iceberg),Map(),None,None,None,false), true","children":[],"metadata":{},"metrics":[]},"time":1714515294867,"modifiedConfigs":{}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":0,"time":1714515295008} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":1,"description":"CREATE TABLE IF NOT EXISTS local.developers (\n username string,\n firstname string,\n lastname string)\nUSING iceberg\nPARTITIONED BY (username)","details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nCommandResult (1)\n +- CreateTable (2)\n\n\n(1) CommandResult\nOutput: []\nArguments: \n\n(2) CreateTable\nArguments: org.apache.iceberg.spark.SparkCatalog@295e9aef, developers, [StructField(username,StringType,true), StructField(firstname,StringType,true), StructField(lastname,StringType,true)], [identity(username)], TableSpec(Map(),Some(iceberg),Map(),None,None,None,false), true\n\n","sparkPlanInfo":{"nodeName":"CommandResult","simpleString":"CommandResult ","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":0,"metricType":"sum"}]},"time":1714515295027,"modifiedConfigs":{}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":1,"time":1714515295042} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":2,"description":"\nCREATE TABLE IF NOT EXISTS local.projects (\n creator string,\n projectname string)\nUSING iceberg\nPARTITIONED BY (creator)","details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nCreateTable (1)\n\n\n(1) CreateTable\nArguments: org.apache.iceberg.spark.SparkCatalog@295e9aef, projects, [StructField(creator,StringType,true), StructField(projectname,StringType,true)], [identity(creator)], TableSpec(Map(),Some(iceberg),Map(),None,None,None,false), true\n\n","sparkPlanInfo":{"nodeName":"CreateTable","simpleString":"CreateTable org.apache.iceberg.spark.SparkCatalog@295e9aef, projects, [StructField(creator,StringType,true), StructField(projectname,StringType,true)], [identity(creator)], TableSpec(Map(),Some(iceberg),Map(),None,None,None,false), true","children":[],"metadata":{},"metrics":[]},"time":1714515295063,"modifiedConfigs":{}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":2,"time":1714515295067} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":3,"description":"\nCREATE TABLE IF NOT EXISTS local.projects (\n creator string,\n projectname string)\nUSING iceberg\nPARTITIONED BY (creator)","details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nCommandResult (1)\n +- CreateTable (2)\n\n\n(1) CommandResult\nOutput: []\nArguments: \n\n(2) CreateTable\nArguments: org.apache.iceberg.spark.SparkCatalog@295e9aef, projects, [StructField(creator,StringType,true), StructField(projectname,StringType,true)], [identity(creator)], TableSpec(Map(),Some(iceberg),Map(),None,None,None,false), true\n\n","sparkPlanInfo":{"nodeName":"CommandResult","simpleString":"CommandResult ","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1,"metricType":"sum"}]},"time":1714515295072,"modifiedConfigs":{}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":3,"time":1714515295072} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":4,"description":"\nINSERT INTO local.developers VALUES(\"krisnova\", \"Kris\", \"Nova\")","details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nAppendData (4)\n+- AdaptiveSparkPlan (3)\n +- Exchange (2)\n +- LocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [3]: [username#6, firstname#7, lastname#8]\nArguments: [username#6, firstname#7, lastname#8]\n\n(2) Exchange\nInput [3]: [username#6, firstname#7, lastname#8]\nArguments: hashpartitioning(username#6, 200), REPARTITION_BY_NUM, [plan_id=21]\n\n(3) AdaptiveSparkPlan\nOutput [3]: [username#6, firstname#7, lastname#8]\nArguments: isFinalPlan=false\n\n(4) AppendData\nInput [3]: [username#6, firstname#7, lastname#8]\nArguments: org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$2253/0x0000775334f5b4c0@205c5cd0, IcebergWrite(table=local.developers, format=PARQUET)\n\n","sparkPlanInfo":{"nodeName":"AppendData","simpleString":"AppendData org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$2253/0x0000775334f5b4c0@205c5cd0, IcebergWrite(table=local.developers, format=PARQUET)","children":[{"nodeName":"AdaptiveSparkPlan","simpleString":"AdaptiveSparkPlan isFinalPlan=false","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(username#6, 200), REPARTITION_BY_NUM, [plan_id=21]","children":[{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [username#6, firstname#7, lastname#8]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":14,"metricType":"sum"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":12,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":13,"metricType":"nsTiming"},{"name":"records read","accumulatorId":10,"metricType":"sum"},{"name":"local bytes read","accumulatorId":8,"metricType":"size"},{"name":"fetch wait time","accumulatorId":9,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":6,"metricType":"size"},{"name":"local blocks read","accumulatorId":5,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":4,"metricType":"sum"},{"name":"data size","accumulatorId":2,"metricType":"size"},{"name":"number of partitions","accumulatorId":3,"metricType":"sum"},{"name":"remote bytes read to disk","accumulatorId":7,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":11,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]},"time":1714515295209,"modifiedConfigs":{}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate","executionId":4,"physicalPlanDescription":"== Physical Plan ==\nAppendData (5)\n+- AdaptiveSparkPlan (4)\n +- == Current Plan ==\n ShuffleQueryStage (3)\n +- Exchange (2)\n +- LocalTableScan (1)\n +- == Initial Plan ==\n Exchange (2)\n +- LocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [3]: [username#6, firstname#7, lastname#8]\nArguments: [username#6, firstname#7, lastname#8]\n\n(2) Exchange\nInput [3]: [username#6, firstname#7, lastname#8]\nArguments: hashpartitioning(username#6, 200), REPARTITION_BY_NUM, [plan_id=21]\n\n(3) ShuffleQueryStage\nOutput [3]: [username#6, firstname#7, lastname#8]\nArguments: 0\n\n(4) AdaptiveSparkPlan\nOutput [3]: [username#6, firstname#7, lastname#8]\nArguments: isFinalPlan=false\n\n(5) AppendData\nInput [3]: [username#6, firstname#7, lastname#8]\nArguments: org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$2253/0x0000775334f5b4c0@205c5cd0, IcebergWrite(table=local.developers, format=PARQUET)\n\n","sparkPlanInfo":{"nodeName":"AppendData","simpleString":"AppendData org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$2253/0x0000775334f5b4c0@205c5cd0, IcebergWrite(table=local.developers, format=PARQUET)","children":[{"nodeName":"AdaptiveSparkPlan","simpleString":"AdaptiveSparkPlan isFinalPlan=false","children":[{"nodeName":"ShuffleQueryStage","simpleString":"ShuffleQueryStage 0","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(username#6, 200), REPARTITION_BY_NUM, [plan_id=21]","children":[{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [username#6, firstname#7, lastname#8]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":14,"metricType":"sum"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":12,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":13,"metricType":"nsTiming"},{"name":"records read","accumulatorId":10,"metricType":"sum"},{"name":"local bytes read","accumulatorId":8,"metricType":"size"},{"name":"fetch wait time","accumulatorId":9,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":6,"metricType":"size"},{"name":"local blocks read","accumulatorId":5,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":4,"metricType":"sum"},{"name":"data size","accumulatorId":2,"metricType":"size"},{"name":"number of partitions","accumulatorId":3,"metricType":"sum"},{"name":"remote bytes read to disk","accumulatorId":7,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":11,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":4,"accumUpdates":[[3,200]]} +{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1714515295457,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"processLine at CliDriver.java:336","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"Exchange\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"LocalTableScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"2\",\"name\":\"LocalTableScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[0],"Properties":{"spark.sql.warehouse.dir":"file:/home/holden/repos/sparklens/spark-warehouse","spark.executor.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.driver.host":"10.69.200.180","spark.eventLog.enabled":"true","spark.driver.port":"37317","spark.sql.catalog.local.type":"hadoop","spark.jars":"","spark.sql.catalog.spark_catalog":"org.apache.iceberg.spark.SparkSessionCatalog","spark.sql.datetime.java8API.enabled":"true","spark.sql.hive.version":"2.3.9","spark.app.name":"fresh","spark.rdd.scope":"{\"id\":\"1\",\"name\":\"Exchange\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.submitTime":"1714515291527","spark.job.description":"\nINSERT INTO local.developers VALUES(\"krisnova\", \"Kris\", \"Nova\")","spark.app.startTime":"1714515291883","spark.executor.id":"driver","spark.sql.catalog.local.warehouse":"/home/holden/repos/sparklens/warehouse","spark.driver.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.sql.catalog.local":"org.apache.iceberg.spark.SparkCatalog","spark.submit.deployMode":"client","spark.master":"local[5]","spark.sql.execution.id":"4","spark.sql.catalogImplementation":"hive","spark.sql.catalog.spark_catalog.type":"hive","spark.app.id":"local-1714515292260"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"processLine at CliDriver.java:336","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"Exchange\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"LocalTableScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"2\",\"name\":\"LocalTableScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1714515295469,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/holden/repos/sparklens/spark-warehouse","spark.executor.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.driver.host":"10.69.200.180","spark.eventLog.enabled":"true","spark.driver.port":"37317","spark.sql.catalog.local.type":"hadoop","spark.jars":"","spark.sql.catalog.spark_catalog":"org.apache.iceberg.spark.SparkSessionCatalog","spark.sql.datetime.java8API.enabled":"true","spark.sql.hive.version":"2.3.9","spark.app.name":"fresh","spark.rdd.scope":"{\"id\":\"1\",\"name\":\"Exchange\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.submitTime":"1714515291527","spark.job.description":"\nINSERT INTO local.developers VALUES(\"krisnova\", \"Kris\", \"Nova\")","spark.app.startTime":"1714515291883","spark.executor.id":"driver","spark.sql.catalog.local.warehouse":"/home/holden/repos/sparklens/warehouse","spark.driver.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.sql.catalog.local":"org.apache.iceberg.spark.SparkCatalog","spark.submit.deployMode":"client","spark.master":"local[5]","spark.sql.execution.id":"4","spark.sql.catalogImplementation":"hive","spark.sql.catalog.spark_catalog.type":"hive","spark.app.id":"local-1714515292260"}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Partition ID":0,"Launch Time":1714515295579,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Partition ID":0,"Launch Time":1714515295579,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295684,"Failed":false,"Killed":false,"Accumulables":[{"ID":2,"Name":"data size","Update":"56","Value":"56","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":11,"Name":"shuffle bytes written","Update":"95","Value":"95","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":12,"Name":"shuffle records written","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":13,"Name":"shuffle write time","Update":"14421171","Value":"14421171","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":19,"Value":19,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":19333175,"Value":19333175,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":59,"Value":59,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":58235555,"Value":58235555,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1828,"Value":1828,"Internal":true,"Count Failed Values":true},{"ID":33,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":95,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":34,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":35,"Name":"internal.metrics.shuffle.write.writeTime","Update":14421171,"Value":14421171,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":19,"Executor Deserialize CPU Time":19333175,"Executor Run Time":59,"Executor CPU Time":58235555,"Peak Execution Memory":0,"Result Size":1828,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":95,"Shuffle Write Time":14421171,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"processLine at CliDriver.java:336","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"Exchange\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"LocalTableScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"2\",\"name\":\"LocalTableScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1714515295469,"Completion Time":1714515295688,"Accumulables":[{"ID":2,"Name":"data size","Value":"56","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":11,"Name":"shuffle bytes written","Value":"95","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":12,"Name":"shuffle records written","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":13,"Name":"shuffle write time","Value":"14421171","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Value":19,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Value":19333175,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Value":59,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Value":58235555,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Value":1828,"Internal":true,"Count Failed Values":true},{"ID":33,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":95,"Internal":true,"Count Failed Values":true},{"ID":34,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":1,"Internal":true,"Count Failed Values":true},{"ID":35,"Name":"internal.metrics.shuffle.write.writeTime","Value":14421171,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1714515295691,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate","executionId":4,"physicalPlanDescription":"== Physical Plan ==\nAppendData (5)\n+- AdaptiveSparkPlan (4)\n +- == Final Plan ==\n ShuffleQueryStage (3), Statistics(sizeInBytes=56.0 B, rowCount=1)\n +- Exchange (2)\n +- LocalTableScan (1)\n +- == Initial Plan ==\n Exchange (2)\n +- LocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [3]: [username#6, firstname#7, lastname#8]\nArguments: [username#6, firstname#7, lastname#8]\n\n(2) Exchange\nInput [3]: [username#6, firstname#7, lastname#8]\nArguments: hashpartitioning(username#6, 200), REPARTITION_BY_NUM, [plan_id=21]\n\n(3) ShuffleQueryStage\nOutput [3]: [username#6, firstname#7, lastname#8]\nArguments: 0\n\n(4) AdaptiveSparkPlan\nOutput [3]: [username#6, firstname#7, lastname#8]\nArguments: isFinalPlan=true\n\n(5) AppendData\nInput [3]: [username#6, firstname#7, lastname#8]\nArguments: org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$2253/0x0000775334f5b4c0@205c5cd0, IcebergWrite(table=local.developers, format=PARQUET)\n\n","sparkPlanInfo":{"nodeName":"AppendData","simpleString":"AppendData org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$2253/0x0000775334f5b4c0@205c5cd0, IcebergWrite(table=local.developers, format=PARQUET)","children":[{"nodeName":"AdaptiveSparkPlan","simpleString":"AdaptiveSparkPlan isFinalPlan=true","children":[{"nodeName":"ShuffleQueryStage","simpleString":"ShuffleQueryStage 0","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(username#6, 200), REPARTITION_BY_NUM, [plan_id=21]","children":[{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [username#6, firstname#7, lastname#8]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":14,"metricType":"sum"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":12,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":13,"metricType":"nsTiming"},{"name":"records read","accumulatorId":10,"metricType":"sum"},{"name":"local bytes read","accumulatorId":8,"metricType":"size"},{"name":"fetch wait time","accumulatorId":9,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":6,"metricType":"size"},{"name":"local blocks read","accumulatorId":5,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":4,"metricType":"sum"},{"name":"data size","accumulatorId":2,"metricType":"size"},{"name":"number of partitions","accumulatorId":3,"metricType":"sum"},{"name":"remote bytes read to disk","accumulatorId":7,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":11,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}} +{"Event":"SparkListenerJobStart","Job ID":1,"Submission Time":1714515295727,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"processLine at CliDriver.java:336","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"Exchange\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"LocalTableScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"2\",\"name\":\"LocalTableScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"processLine at CliDriver.java:336","Number of Tasks":200,"RDD Info":[{"RDD ID":3,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"4\",\"name\":\"Exchange\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"UNORDERED","Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[1],"Details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[1,2],"Properties":{"spark.sql.warehouse.dir":"file:/home/holden/repos/sparklens/spark-warehouse","spark.executor.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.driver.host":"10.69.200.180","spark.eventLog.enabled":"true","spark.driver.port":"37317","__fetch_continuous_blocks_in_batch_enabled":"true","spark.sql.catalog.local.type":"hadoop","spark.jars":"","spark.sql.catalog.spark_catalog":"org.apache.iceberg.spark.SparkSessionCatalog","spark.sql.datetime.java8API.enabled":"true","spark.sql.hive.version":"2.3.9","spark.app.name":"fresh","spark.submit.pyFiles":"","spark.app.submitTime":"1714515291527","spark.job.description":"\nINSERT INTO local.developers VALUES(\"krisnova\", \"Kris\", \"Nova\")","spark.app.startTime":"1714515291883","spark.executor.id":"driver","spark.sql.catalog.local.warehouse":"/home/holden/repos/sparklens/warehouse","spark.driver.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.sql.catalog.local":"org.apache.iceberg.spark.SparkCatalog","spark.submit.deployMode":"client","spark.master":"local[5]","spark.sql.execution.id":"4","spark.sql.catalogImplementation":"hive","spark.sql.catalog.spark_catalog.type":"hive","spark.app.id":"local-1714515292260"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"processLine at CliDriver.java:336","Number of Tasks":200,"RDD Info":[{"RDD ID":3,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"4\",\"name\":\"Exchange\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"UNORDERED","Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[1],"Details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1714515295739,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/holden/repos/sparklens/spark-warehouse","spark.executor.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.driver.host":"10.69.200.180","spark.eventLog.enabled":"true","spark.driver.port":"37317","__fetch_continuous_blocks_in_batch_enabled":"true","spark.sql.catalog.local.type":"hadoop","spark.jars":"","spark.sql.catalog.spark_catalog":"org.apache.iceberg.spark.SparkSessionCatalog","spark.sql.datetime.java8API.enabled":"true","spark.sql.hive.version":"2.3.9","spark.app.name":"fresh","spark.submit.pyFiles":"","spark.app.submitTime":"1714515291527","spark.job.description":"\nINSERT INTO local.developers VALUES(\"krisnova\", \"Kris\", \"Nova\")","spark.app.startTime":"1714515291883","spark.executor.id":"driver","spark.sql.catalog.local.warehouse":"/home/holden/repos/sparklens/warehouse","spark.driver.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.sql.catalog.local":"org.apache.iceberg.spark.SparkCatalog","spark.submit.deployMode":"client","spark.master":"local[5]","spark.sql.execution.id":"4","spark.sql.catalogImplementation":"hive","spark.sql.catalog.spark_catalog.type":"hive","spark.app.id":"local-1714515292260"}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":68,"Attempt":0,"Partition ID":68,"Launch Time":1714515295757,"Executor ID":"driver","Host":"10.69.200.180","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":0,"Attempt":0,"Partition ID":0,"Launch Time":1714515295759,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":1,"Attempt":0,"Partition ID":1,"Launch Time":1714515295760,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":2,"Attempt":0,"Partition ID":2,"Launch Time":1714515295760,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":3,"Attempt":0,"Partition ID":3,"Launch Time":1714515295760,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":4,"Attempt":0,"Partition ID":4,"Launch Time":1714515295807,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":1,"Attempt":0,"Partition ID":1,"Launch Time":1714515295760,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295808,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":9,"Value":9,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":9020172,"Value":9020172,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":33,"Value":33,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":16902125,"Value":16902125,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2556,"Value":2556,"Internal":true,"Count Failed Values":true},{"ID":46,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":9,"Executor Deserialize CPU Time":9020172,"Executor Run Time":33,"Executor CPU Time":16902125,"Peak Execution Memory":0,"Result Size":2556,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":5,"Attempt":0,"Partition ID":5,"Launch Time":1714515295808,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":2,"Attempt":0,"Partition ID":2,"Launch Time":1714515295760,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295809,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":9,"Value":18,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8300586,"Value":17320758,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":33,"Value":66,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":16607497,"Value":33509622,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2556,"Value":5112,"Internal":true,"Count Failed Values":true},{"ID":46,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":9,"Executor Deserialize CPU Time":8300586,"Executor Run Time":33,"Executor CPU Time":16607497,"Peak Execution Memory":0,"Result Size":2556,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":6,"Attempt":0,"Partition ID":6,"Launch Time":1714515295809,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":0,"Attempt":0,"Partition ID":0,"Launch Time":1714515295759,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295809,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":9,"Value":27,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8280251,"Value":25601009,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":33,"Value":99,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":20868918,"Value":54378540,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2556,"Value":7668,"Internal":true,"Count Failed Values":true},{"ID":46,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":9,"Executor Deserialize CPU Time":8280251,"Executor Run Time":33,"Executor CPU Time":20868918,"Peak Execution Memory":0,"Result Size":2556,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":7,"Attempt":0,"Partition ID":7,"Launch Time":1714515295810,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":3,"Attempt":0,"Partition ID":3,"Launch Time":1714515295760,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295810,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":8,"Value":35,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7678251,"Value":33279260,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":33,"Value":132,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":16234437,"Value":70612977,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2556,"Value":10224,"Internal":true,"Count Failed Values":true},{"ID":46,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":4,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":8,"Executor Deserialize CPU Time":7678251,"Executor Run Time":33,"Executor CPU Time":16234437,"Peak Execution Memory":0,"Result Size":2556,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":8,"Attempt":0,"Partition ID":8,"Launch Time":1714515295814,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":9,"Attempt":0,"Partition ID":9,"Launch Time":1714515295814,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":6,"Attempt":0,"Partition ID":6,"Launch Time":1714515295809,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295815,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":37,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2113672,"Value":35392932,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":133,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1638895,"Value":72251872,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":12737,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":2113672,"Executor Run Time":1,"Executor CPU Time":1638895,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":5,"Attempt":0,"Partition ID":5,"Launch Time":1714515295808,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295815,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":39,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2730780,"Value":38123712,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":134,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1762828,"Value":74014700,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":15250,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":2730780,"Executor Run Time":1,"Executor CPU Time":1762828,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":10,"Attempt":0,"Partition ID":10,"Launch Time":1714515295815,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":7,"Attempt":0,"Partition ID":7,"Launch Time":1714515295810,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295815,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":40,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1758840,"Value":39882552,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":135,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1786269,"Value":75800969,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":17763,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1758840,"Executor Run Time":1,"Executor CPU Time":1786269,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":11,"Attempt":0,"Partition ID":11,"Launch Time":1714515295816,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":4,"Attempt":0,"Partition ID":4,"Launch Time":1714515295807,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295816,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":43,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3594385,"Value":43476937,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":137,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":2404117,"Value":78205086,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":20276,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3594385,"Executor Run Time":2,"Executor CPU Time":2404117,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":12,"Attempt":0,"Partition ID":12,"Launch Time":1714515295819,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":13,"Attempt":0,"Partition ID":13,"Launch Time":1714515295819,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":8,"Attempt":0,"Partition ID":8,"Launch Time":1714515295814,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295820,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":44,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1433408,"Value":44910345,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":138,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1728581,"Value":79933667,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":22789,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1433408,"Executor Run Time":1,"Executor CPU Time":1728581,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":9,"Attempt":0,"Partition ID":9,"Launch Time":1714515295814,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295820,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":45,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1639795,"Value":46550140,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":139,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1860024,"Value":81793691,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":25302,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1639795,"Executor Run Time":1,"Executor CPU Time":1860024,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":14,"Attempt":0,"Partition ID":14,"Launch Time":1714515295821,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":11,"Attempt":0,"Partition ID":11,"Launch Time":1714515295816,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295821,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":47,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2696621,"Value":49246761,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":140,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1356750,"Value":83150441,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":27815,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":2696621,"Executor Run Time":1,"Executor CPU Time":1356750,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":15,"Attempt":0,"Partition ID":15,"Launch Time":1714515295823,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":10,"Attempt":0,"Partition ID":10,"Launch Time":1714515295815,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295823,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":50,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3232284,"Value":52479045,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":142,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":2319311,"Value":85469752,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":30328,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3232284,"Executor Run Time":2,"Executor CPU Time":2319311,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":16,"Attempt":0,"Partition ID":16,"Launch Time":1714515295824,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":12,"Attempt":0,"Partition ID":12,"Launch Time":1714515295819,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295824,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":51,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1300712,"Value":53779757,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":143,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1329212,"Value":86798964,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":32841,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1300712,"Executor Run Time":1,"Executor CPU Time":1329212,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":17,"Attempt":0,"Partition ID":17,"Launch Time":1714515295825,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":13,"Attempt":0,"Partition ID":13,"Launch Time":1714515295819,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295825,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":52,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1486392,"Value":55266149,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":144,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1269474,"Value":88068438,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":35354,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1486392,"Executor Run Time":1,"Executor CPU Time":1269474,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":18,"Attempt":0,"Partition ID":18,"Launch Time":1714515295825,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":14,"Attempt":0,"Partition ID":14,"Launch Time":1714515295821,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295826,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":54,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1641564,"Value":56907713,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":145,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1099303,"Value":89167741,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":37867,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1641564,"Executor Run Time":1,"Executor CPU Time":1099303,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":19,"Attempt":0,"Partition ID":19,"Launch Time":1714515295827,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":20,"Attempt":0,"Partition ID":20,"Launch Time":1714515295827,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":15,"Attempt":0,"Partition ID":15,"Launch Time":1714515295823,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295828,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":55,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1173569,"Value":58081282,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":146,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1025958,"Value":90193699,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":40380,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1173569,"Executor Run Time":1,"Executor CPU Time":1025958,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":16,"Attempt":0,"Partition ID":16,"Launch Time":1714515295824,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295828,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":56,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1256766,"Value":59338048,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":147,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1111483,"Value":91305182,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":42893,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1256766,"Executor Run Time":1,"Executor CPU Time":1111483,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":21,"Attempt":0,"Partition ID":21,"Launch Time":1714515295829,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":18,"Attempt":0,"Partition ID":18,"Launch Time":1714515295825,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295829,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":57,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1406908,"Value":60744956,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":148,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1092834,"Value":92398016,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":45406,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1406908,"Executor Run Time":1,"Executor CPU Time":1092834,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":22,"Attempt":0,"Partition ID":22,"Launch Time":1714515295830,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":19,"Attempt":0,"Partition ID":19,"Launch Time":1714515295827,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295831,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":58,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1016218,"Value":61761174,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":149,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1184279,"Value":93582295,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":47919,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1016218,"Executor Run Time":1,"Executor CPU Time":1184279,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":23,"Attempt":0,"Partition ID":23,"Launch Time":1714515295831,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":17,"Attempt":0,"Partition ID":17,"Launch Time":1714515295825,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295831,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2233806,"Value":63994980,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":151,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":2160338,"Value":95742633,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":50432,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":2233806,"Executor Run Time":2,"Executor CPU Time":2160338,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":24,"Attempt":0,"Partition ID":24,"Launch Time":1714515295832,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":20,"Attempt":0,"Partition ID":20,"Launch Time":1714515295827,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295832,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":816228,"Value":64811208,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":152,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1003477,"Value":96746110,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":52902,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":816228,"Executor Run Time":1,"Executor CPU Time":1003477,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":25,"Attempt":0,"Partition ID":25,"Launch Time":1714515295832,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":21,"Attempt":0,"Partition ID":21,"Launch Time":1714515295829,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295833,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":846427,"Value":65657635,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":153,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1019247,"Value":97765357,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":55372,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":846427,"Executor Run Time":1,"Executor CPU Time":1019247,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":26,"Attempt":0,"Partition ID":26,"Launch Time":1714515295833,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":22,"Attempt":0,"Partition ID":22,"Launch Time":1714515295830,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295834,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":863215,"Value":66520850,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":154,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1026677,"Value":98792034,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":57842,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":863215,"Executor Run Time":1,"Executor CPU Time":1026677,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":27,"Attempt":0,"Partition ID":27,"Launch Time":1714515295836,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":24,"Attempt":0,"Partition ID":24,"Launch Time":1714515295832,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295836,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":61,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1529192,"Value":68050042,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":155,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1578640,"Value":100370674,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":60355,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1529192,"Executor Run Time":1,"Executor CPU Time":1578640,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":28,"Attempt":0,"Partition ID":28,"Launch Time":1714515295837,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":25,"Attempt":0,"Partition ID":25,"Launch Time":1714515295832,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295837,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":62,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1297193,"Value":69347235,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":156,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1713762,"Value":102084436,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":62868,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1297193,"Executor Run Time":1,"Executor CPU Time":1713762,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":29,"Attempt":0,"Partition ID":29,"Launch Time":1714515295845,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":26,"Attempt":0,"Partition ID":26,"Launch Time":1714515295833,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295846,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1203609,"Value":70550844,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":157,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1392829,"Value":103477265,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":65381,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1203609,"Executor Run Time":1,"Executor CPU Time":1392829,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":32,"Index":30,"Attempt":0,"Partition ID":30,"Launch Time":1714515295846,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":23,"Attempt":0,"Partition ID":23,"Launch Time":1714515295831,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295846,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":65,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2441155,"Value":72991999,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":159,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":2508771,"Value":105986036,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":67894,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":2441155,"Executor Run Time":2,"Executor CPU Time":2508771,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":33,"Index":31,"Attempt":0,"Partition ID":31,"Launch Time":1714515295847,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":27,"Attempt":0,"Partition ID":27,"Launch Time":1714515295836,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295848,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":8,"Value":73,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1424737,"Value":74416736,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":161,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":2084764,"Value":108070800,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2556,"Value":70450,"Internal":true,"Count Failed Values":true},{"ID":45,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":6,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":8,"Executor Deserialize CPU Time":1424737,"Executor Run Time":2,"Executor CPU Time":2084764,"Peak Execution Memory":0,"Result Size":2556,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":34,"Index":32,"Attempt":0,"Partition ID":32,"Launch Time":1714515295848,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":28,"Attempt":0,"Partition ID":28,"Launch Time":1714515295837,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295849,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":74,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1611278,"Value":76028014,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":162,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1792548,"Value":109863348,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":72963,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1611278,"Executor Run Time":1,"Executor CPU Time":1792548,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":35,"Index":33,"Attempt":0,"Partition ID":33,"Launch Time":1714515295850,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":32,"Index":30,"Attempt":0,"Partition ID":30,"Launch Time":1714515295846,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295850,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":75,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1489572,"Value":77517586,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":163,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1515903,"Value":111379251,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":75476,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1489572,"Executor Run Time":1,"Executor CPU Time":1515903,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":36,"Index":34,"Attempt":0,"Partition ID":34,"Launch Time":1714515295851,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":33,"Index":31,"Attempt":0,"Partition ID":31,"Launch Time":1714515295847,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295852,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":999868,"Value":78517454,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":164,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1723222,"Value":113102473,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":77946,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":999868,"Executor Run Time":1,"Executor CPU Time":1723222,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":37,"Index":35,"Attempt":0,"Partition ID":35,"Launch Time":1714515295852,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":31,"Index":29,"Attempt":0,"Partition ID":29,"Launch Time":1714515295845,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295852,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":77,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2320470,"Value":80837924,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":166,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":2579528,"Value":115682001,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":80459,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":2320470,"Executor Run Time":2,"Executor CPU Time":2579528,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":38,"Index":36,"Attempt":0,"Partition ID":36,"Launch Time":1714515295852,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":34,"Index":32,"Attempt":0,"Partition ID":32,"Launch Time":1714515295848,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295853,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":771611,"Value":81609535,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":167,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1550240,"Value":117232241,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":82929,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":771611,"Executor Run Time":1,"Executor CPU Time":1550240,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":39,"Index":37,"Attempt":0,"Partition ID":37,"Launch Time":1714515295855,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":35,"Index":33,"Attempt":0,"Partition ID":33,"Launch Time":1714515295850,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295855,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":78,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1466694,"Value":83076229,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":169,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1921754,"Value":119153995,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":85442,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1466694,"Executor Run Time":2,"Executor CPU Time":1921754,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":40,"Index":38,"Attempt":0,"Partition ID":38,"Launch Time":1714515295857,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":38,"Index":36,"Attempt":0,"Partition ID":36,"Launch Time":1714515295852,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295857,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":79,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1396899,"Value":84473128,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":170,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1656054,"Value":120810049,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":87955,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1396899,"Executor Run Time":1,"Executor CPU Time":1656054,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":41,"Index":39,"Attempt":0,"Partition ID":39,"Launch Time":1714515295857,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":36,"Index":34,"Attempt":0,"Partition ID":34,"Launch Time":1714515295851,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295858,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":80,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1875524,"Value":86348652,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":172,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":2497510,"Value":123307559,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":90468,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1875524,"Executor Run Time":2,"Executor CPU Time":2497510,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":42,"Index":40,"Attempt":0,"Partition ID":40,"Launch Time":1714515295858,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":37,"Index":35,"Attempt":0,"Partition ID":35,"Launch Time":1714515295852,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295858,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":81,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1600697,"Value":87949349,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":174,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1972369,"Value":125279928,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":92981,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1600697,"Executor Run Time":2,"Executor CPU Time":1972369,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":43,"Index":41,"Attempt":0,"Partition ID":41,"Launch Time":1714515295859,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":39,"Index":37,"Attempt":0,"Partition ID":37,"Launch Time":1714515295855,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295859,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":82,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1161921,"Value":89111270,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":175,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1465334,"Value":126745262,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":95494,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1161921,"Executor Run Time":1,"Executor CPU Time":1465334,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":44,"Index":42,"Attempt":0,"Partition ID":42,"Launch Time":1714515295862,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":42,"Index":40,"Attempt":0,"Partition ID":40,"Launch Time":1714515295858,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295863,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":83,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1203908,"Value":90315178,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":176,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1506951,"Value":128252213,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":98007,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1203908,"Executor Run Time":1,"Executor CPU Time":1506951,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":45,"Index":43,"Attempt":0,"Partition ID":43,"Launch Time":1714515295863,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":40,"Index":38,"Attempt":0,"Partition ID":38,"Launch Time":1714515295857,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295863,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1707432,"Value":92022610,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":178,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":2211784,"Value":130463997,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":100520,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1707432,"Executor Run Time":2,"Executor CPU Time":2211784,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":46,"Index":44,"Attempt":0,"Partition ID":44,"Launch Time":1714515295864,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":41,"Index":39,"Attempt":0,"Partition ID":39,"Launch Time":1714515295857,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295864,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":85,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1606947,"Value":93629557,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":180,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1840994,"Value":132304991,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":103033,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1606947,"Executor Run Time":2,"Executor CPU Time":1840994,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":47,"Index":45,"Attempt":0,"Partition ID":45,"Launch Time":1714515295865,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":43,"Index":41,"Attempt":0,"Partition ID":41,"Launch Time":1714515295859,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295865,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":86,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1075236,"Value":94704793,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":181,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1452925,"Value":133757916,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":105546,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1075236,"Executor Run Time":1,"Executor CPU Time":1452925,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":48,"Index":46,"Attempt":0,"Partition ID":46,"Launch Time":1714515295866,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":44,"Index":42,"Attempt":0,"Partition ID":42,"Launch Time":1714515295862,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295867,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":87,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1429097,"Value":96133890,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":182,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1540382,"Value":135298298,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":108059,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1429097,"Executor Run Time":1,"Executor CPU Time":1540382,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":49,"Index":47,"Attempt":0,"Partition ID":47,"Launch Time":1714515295867,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":45,"Index":43,"Attempt":0,"Partition ID":43,"Launch Time":1714515295863,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295868,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":88,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1239666,"Value":97373556,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":183,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1613917,"Value":136912215,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":110572,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1239666,"Executor Run Time":1,"Executor CPU Time":1613917,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":50,"Index":48,"Attempt":0,"Partition ID":48,"Launch Time":1714515295869,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":51,"Index":49,"Attempt":0,"Partition ID":49,"Launch Time":1714515295869,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":46,"Index":44,"Attempt":0,"Partition ID":44,"Launch Time":1714515295864,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295870,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":89,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1518001,"Value":98891557,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":184,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1724381,"Value":138636596,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":113085,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1518001,"Executor Run Time":1,"Executor CPU Time":1724381,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":47,"Index":45,"Attempt":0,"Partition ID":45,"Launch Time":1714515295865,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295870,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":90,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1242856,"Value":100134413,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":185,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1518162,"Value":140154758,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":115598,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1242856,"Executor Run Time":1,"Executor CPU Time":1518162,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":52,"Index":50,"Attempt":0,"Partition ID":50,"Launch Time":1714515295871,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":48,"Index":46,"Attempt":0,"Partition ID":46,"Launch Time":1714515295866,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295871,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":91,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1010138,"Value":101144551,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":186,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1603626,"Value":141758384,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":118111,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1010138,"Executor Run Time":1,"Executor CPU Time":1603626,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":53,"Index":51,"Attempt":0,"Partition ID":51,"Launch Time":1714515295871,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":49,"Index":47,"Attempt":0,"Partition ID":47,"Launch Time":1714515295867,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295872,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":92,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1429407,"Value":102573958,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":187,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1432206,"Value":143190590,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":120624,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1429407,"Executor Run Time":1,"Executor CPU Time":1432206,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":54,"Index":52,"Attempt":0,"Partition ID":52,"Launch Time":1714515295873,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":50,"Index":48,"Attempt":0,"Partition ID":48,"Launch Time":1714515295869,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295874,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":93,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1399549,"Value":103973507,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":188,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1370228,"Value":144560818,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":123137,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1399549,"Executor Run Time":1,"Executor CPU Time":1370228,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":55,"Index":53,"Attempt":0,"Partition ID":53,"Launch Time":1714515295874,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":51,"Index":49,"Attempt":0,"Partition ID":49,"Launch Time":1714515295869,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295874,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":94,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1592538,"Value":105566045,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":190,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1932830,"Value":146493648,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":125650,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1592538,"Executor Run Time":2,"Executor CPU Time":1932830,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":56,"Index":54,"Attempt":0,"Partition ID":54,"Launch Time":1714515295875,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":52,"Index":50,"Attempt":0,"Partition ID":50,"Launch Time":1714515295871,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295875,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1062435,"Value":106628480,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":191,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1410181,"Value":147903829,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":128163,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1062435,"Executor Run Time":1,"Executor CPU Time":1410181,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":57,"Index":55,"Attempt":0,"Partition ID":55,"Launch Time":1714515295875,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":53,"Index":51,"Attempt":0,"Partition ID":51,"Launch Time":1714515295871,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295876,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":922991,"Value":107551471,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":192,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1158050,"Value":149061879,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":130633,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":922991,"Executor Run Time":1,"Executor CPU Time":1158050,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":58,"Index":56,"Attempt":0,"Partition ID":56,"Launch Time":1714515295877,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":54,"Index":52,"Attempt":0,"Partition ID":52,"Launch Time":1714515295873,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295877,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":864426,"Value":108415897,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":193,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1128802,"Value":150190681,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":133103,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":864426,"Executor Run Time":1,"Executor CPU Time":1128802,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":59,"Index":57,"Attempt":0,"Partition ID":57,"Launch Time":1714515295877,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":55,"Index":53,"Attempt":0,"Partition ID":53,"Launch Time":1714515295874,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295878,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":96,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":944181,"Value":109360078,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":194,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1028628,"Value":151219309,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":135616,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":944181,"Executor Run Time":1,"Executor CPU Time":1028628,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":60,"Index":58,"Attempt":0,"Partition ID":58,"Launch Time":1714515295878,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":56,"Index":54,"Attempt":0,"Partition ID":54,"Launch Time":1714515295875,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295878,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":97,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1100363,"Value":110460441,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":195,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1102683,"Value":152321992,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":138129,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1100363,"Executor Run Time":1,"Executor CPU Time":1102683,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":61,"Index":59,"Attempt":0,"Partition ID":59,"Launch Time":1714515295880,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":62,"Index":60,"Attempt":0,"Partition ID":60,"Launch Time":1714515295880,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":58,"Index":56,"Attempt":0,"Partition ID":56,"Launch Time":1714515295877,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295880,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":98,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1007929,"Value":111468370,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":196,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1032266,"Value":153354258,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":140642,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1007929,"Executor Run Time":1,"Executor CPU Time":1032266,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":59,"Index":57,"Attempt":0,"Partition ID":57,"Launch Time":1714515295877,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295880,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":752942,"Value":112221312,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":918572,"Value":154272830,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":143069,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":752942,"Executor Run Time":0,"Executor CPU Time":918572,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":63,"Index":61,"Attempt":0,"Partition ID":61,"Launch Time":1714515295880,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":57,"Index":55,"Attempt":0,"Partition ID":55,"Launch Time":1714515295875,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295881,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":99,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1536899,"Value":113758211,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":197,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1688354,"Value":155961184,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":145582,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1536899,"Executor Run Time":1,"Executor CPU Time":1688354,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":64,"Index":62,"Attempt":0,"Partition ID":62,"Launch Time":1714515295881,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":60,"Index":58,"Attempt":0,"Partition ID":58,"Launch Time":1714515295878,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295882,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":837337,"Value":114595548,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":198,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1081594,"Value":157042778,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":148052,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":837337,"Executor Run Time":1,"Executor CPU Time":1081594,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":65,"Index":63,"Attempt":0,"Partition ID":63,"Launch Time":1714515295883,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":61,"Index":59,"Attempt":0,"Partition ID":59,"Launch Time":1714515295880,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295883,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":100,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1054505,"Value":115650053,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":199,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1102664,"Value":158145442,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":150565,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1054505,"Executor Run Time":1,"Executor CPU Time":1102664,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":66,"Index":64,"Attempt":0,"Partition ID":64,"Launch Time":1714515295884,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":63,"Index":61,"Attempt":0,"Partition ID":61,"Launch Time":1714515295880,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295884,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":817839,"Value":116467892,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":826056,"Value":158971498,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":152992,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":817839,"Executor Run Time":0,"Executor CPU Time":826056,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":67,"Index":65,"Attempt":0,"Partition ID":65,"Launch Time":1714515295884,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":62,"Index":60,"Attempt":0,"Partition ID":60,"Launch Time":1714515295880,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295885,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":101,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1594887,"Value":118062779,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":200,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1783629,"Value":160755127,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":155505,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1594887,"Executor Run Time":1,"Executor CPU Time":1783629,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":64,"Index":62,"Attempt":0,"Partition ID":62,"Launch Time":1714515295881,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295885,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":102,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":933502,"Value":118996281,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":201,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1116322,"Value":161871449,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":158018,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":933502,"Executor Run Time":1,"Executor CPU Time":1116322,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":68,"Index":66,"Attempt":0,"Partition ID":66,"Launch Time":1714515295886,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":69,"Index":67,"Attempt":0,"Partition ID":67,"Launch Time":1714515295887,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":65,"Index":63,"Attempt":0,"Partition ID":63,"Launch Time":1714515295883,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295887,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":103,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":798088,"Value":119794369,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":202,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1293034,"Value":163164483,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":160531,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":798088,"Executor Run Time":1,"Executor CPU Time":1293034,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":70,"Index":69,"Attempt":0,"Partition ID":69,"Launch Time":1714515295887,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":66,"Index":64,"Attempt":0,"Partition ID":64,"Launch Time":1714515295884,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295888,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":745332,"Value":120539701,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":203,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1302043,"Value":164466526,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":163001,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":745332,"Executor Run Time":1,"Executor CPU Time":1302043,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":71,"Index":70,"Attempt":0,"Partition ID":70,"Launch Time":1714515295888,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":67,"Index":65,"Attempt":0,"Partition ID":65,"Launch Time":1714515295884,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295888,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":850527,"Value":121390228,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":204,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":957091,"Value":165423617,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":165471,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":850527,"Executor Run Time":1,"Executor CPU Time":957091,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":72,"Index":71,"Attempt":0,"Partition ID":71,"Launch Time":1714515295890,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":69,"Index":67,"Attempt":0,"Partition ID":67,"Launch Time":1714515295887,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295890,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":104,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1009757,"Value":122399985,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":205,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1408618,"Value":166832235,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":167984,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1009757,"Executor Run Time":1,"Executor CPU Time":1408618,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":73,"Index":72,"Attempt":0,"Partition ID":72,"Launch Time":1714515295891,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":70,"Index":69,"Attempt":0,"Partition ID":69,"Launch Time":1714515295887,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295891,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":960740,"Value":123360725,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":206,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1161841,"Value":167994076,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":170454,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":960740,"Executor Run Time":1,"Executor CPU Time":1161841,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":74,"Index":73,"Attempt":0,"Partition ID":73,"Launch Time":1714515295891,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":68,"Index":66,"Attempt":0,"Partition ID":66,"Launch Time":1714515295886,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295892,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":105,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1455096,"Value":124815821,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":208,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":2098981,"Value":170093057,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":172967,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1455096,"Executor Run Time":2,"Executor CPU Time":2098981,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":75,"Index":74,"Attempt":0,"Partition ID":74,"Launch Time":1714515295892,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":71,"Index":70,"Attempt":0,"Partition ID":70,"Launch Time":1714515295888,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295892,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":895584,"Value":125711405,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":209,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":996368,"Value":171089425,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":175437,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":895584,"Executor Run Time":1,"Executor CPU Time":996368,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":76,"Index":75,"Attempt":0,"Partition ID":75,"Launch Time":1714515295893,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":72,"Index":71,"Attempt":0,"Partition ID":71,"Launch Time":1714515295890,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295893,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":857066,"Value":126568471,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":874874,"Value":171964299,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":177864,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":857066,"Executor Run Time":0,"Executor CPU Time":874874,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":77,"Index":76,"Attempt":0,"Partition ID":76,"Launch Time":1714515295894,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":73,"Index":72,"Attempt":0,"Partition ID":72,"Launch Time":1714515295891,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295894,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":989689,"Value":127558160,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":907434,"Value":172871733,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":180291,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":989689,"Executor Run Time":0,"Executor CPU Time":907434,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":78,"Index":77,"Attempt":0,"Partition ID":77,"Launch Time":1714515295895,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":75,"Index":74,"Attempt":0,"Partition ID":74,"Launch Time":1714515295892,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295895,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":106,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1138052,"Value":128696212,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":210,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1022827,"Value":173894560,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":182804,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1138052,"Executor Run Time":1,"Executor CPU Time":1022827,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":79,"Index":78,"Attempt":0,"Partition ID":78,"Launch Time":1714515295896,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":76,"Index":75,"Attempt":0,"Partition ID":75,"Launch Time":1714515295893,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295896,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":846676,"Value":129542888,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":933533,"Value":174828093,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":185231,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":846676,"Executor Run Time":0,"Executor CPU Time":933533,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":80,"Index":79,"Attempt":0,"Partition ID":79,"Launch Time":1714515295897,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":74,"Index":73,"Attempt":0,"Partition ID":73,"Launch Time":1714515295891,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295897,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":107,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1921780,"Value":131464668,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":211,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1878354,"Value":176706447,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":187744,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1921780,"Executor Run Time":1,"Executor CPU Time":1878354,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":81,"Index":80,"Attempt":0,"Partition ID":80,"Launch Time":1714515295897,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":77,"Index":76,"Attempt":0,"Partition ID":76,"Launch Time":1714515295894,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295897,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":852696,"Value":132317364,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1008339,"Value":177714786,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":190171,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":852696,"Executor Run Time":0,"Executor CPU Time":1008339,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":82,"Index":81,"Attempt":0,"Partition ID":81,"Launch Time":1714515295898,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":78,"Index":77,"Attempt":0,"Partition ID":77,"Launch Time":1714515295895,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295898,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":655136,"Value":132972500,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":779040,"Value":178493826,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":192598,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":655136,"Executor Run Time":0,"Executor CPU Time":779040,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":83,"Index":82,"Attempt":0,"Partition ID":82,"Launch Time":1714515295899,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":79,"Index":78,"Attempt":0,"Partition ID":78,"Launch Time":1714515295896,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295899,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":813559,"Value":133786059,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":212,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1025846,"Value":179519672,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":195068,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":813559,"Executor Run Time":1,"Executor CPU Time":1025846,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":84,"Index":83,"Attempt":0,"Partition ID":83,"Launch Time":1714515295899,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":80,"Index":79,"Attempt":0,"Partition ID":79,"Launch Time":1714515295897,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295899,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":681825,"Value":134467884,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":725203,"Value":180244875,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":197495,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":681825,"Executor Run Time":0,"Executor CPU Time":725203,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":85,"Index":84,"Attempt":0,"Partition ID":84,"Launch Time":1714515295900,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":81,"Index":80,"Attempt":0,"Partition ID":80,"Launch Time":1714515295897,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295900,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":687554,"Value":135155438,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":213,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":829919,"Value":181074794,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":199965,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":687554,"Executor Run Time":1,"Executor CPU Time":829919,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":86,"Index":85,"Attempt":0,"Partition ID":85,"Launch Time":1714515295902,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":84,"Index":83,"Attempt":0,"Partition ID":83,"Launch Time":1714515295899,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295902,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":770680,"Value":135926118,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":901664,"Value":181976458,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":202392,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":770680,"Executor Run Time":0,"Executor CPU Time":901664,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":87,"Index":86,"Attempt":0,"Partition ID":86,"Launch Time":1714515295903,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":88,"Index":87,"Attempt":0,"Partition ID":87,"Launch Time":1714515295903,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":83,"Index":82,"Attempt":0,"Partition ID":82,"Launch Time":1714515295899,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295903,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":108,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1215319,"Value":137141437,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":214,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1018877,"Value":182995335,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":204905,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1215319,"Executor Run Time":1,"Executor CPU Time":1018877,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":85,"Index":84,"Attempt":0,"Partition ID":84,"Launch Time":1714515295900,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295903,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":804368,"Value":137945805,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":215,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":979128,"Value":183974463,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":207375,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":804368,"Executor Run Time":1,"Executor CPU Time":979128,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":89,"Index":88,"Attempt":0,"Partition ID":88,"Launch Time":1714515295904,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":82,"Index":81,"Attempt":0,"Partition ID":81,"Launch Time":1714515295898,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295904,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":110,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2043064,"Value":139988869,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":217,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":2044734,"Value":186019197,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":209888,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":2043064,"Executor Run Time":2,"Executor CPU Time":2044734,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":90,"Index":89,"Attempt":0,"Partition ID":89,"Launch Time":1714515295905,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":91,"Index":90,"Attempt":0,"Partition ID":90,"Launch Time":1714515295906,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":87,"Index":86,"Attempt":0,"Partition ID":86,"Launch Time":1714515295903,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295906,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":111,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1018367,"Value":141007236,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":218,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":888244,"Value":186907441,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":212401,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1018367,"Executor Run Time":1,"Executor CPU Time":888244,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":92,"Index":91,"Attempt":0,"Partition ID":91,"Launch Time":1714515295906,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":86,"Index":85,"Attempt":0,"Partition ID":85,"Launch Time":1714515295902,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295906,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":112,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1480153,"Value":142487389,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":219,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1019768,"Value":187927209,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":214914,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1480153,"Executor Run Time":1,"Executor CPU Time":1019768,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":89,"Index":88,"Attempt":0,"Partition ID":88,"Launch Time":1714515295904,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295906,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":845276,"Value":143332665,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":754822,"Value":188682031,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":217341,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":845276,"Executor Run Time":0,"Executor CPU Time":754822,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":93,"Index":92,"Attempt":0,"Partition ID":92,"Launch Time":1714515295908,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":88,"Index":87,"Attempt":0,"Partition ID":87,"Launch Time":1714515295903,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295908,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":113,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1518232,"Value":144850897,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":221,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":2256723,"Value":190938754,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":219854,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1518232,"Executor Run Time":2,"Executor CPU Time":2256723,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":94,"Index":93,"Attempt":0,"Partition ID":93,"Launch Time":1714515295909,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":91,"Index":90,"Attempt":0,"Partition ID":90,"Launch Time":1714515295906,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295909,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":696394,"Value":145547291,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":888084,"Value":191826838,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":222281,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":696394,"Executor Run Time":0,"Executor CPU Time":888084,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":95,"Index":94,"Attempt":0,"Partition ID":94,"Launch Time":1714515295909,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":90,"Index":89,"Attempt":0,"Partition ID":89,"Launch Time":1714515295905,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295909,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":947661,"Value":146494952,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":965051,"Value":192791889,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":224708,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":947661,"Executor Run Time":0,"Executor CPU Time":965051,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":96,"Index":95,"Attempt":0,"Partition ID":95,"Launch Time":1714515295910,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":92,"Index":91,"Attempt":0,"Partition ID":91,"Launch Time":1714515295906,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295910,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":917513,"Value":147412465,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":849036,"Value":193640925,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":227135,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":917513,"Executor Run Time":0,"Executor CPU Time":849036,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":97,"Index":96,"Attempt":0,"Partition ID":96,"Launch Time":1714515295911,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":98,"Index":97,"Attempt":0,"Partition ID":97,"Launch Time":1714515295911,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":94,"Index":93,"Attempt":0,"Partition ID":93,"Launch Time":1714515295909,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295911,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":649646,"Value":148062111,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":802929,"Value":194443854,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":229562,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":649646,"Executor Run Time":0,"Executor CPU Time":802929,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":99,"Index":98,"Attempt":0,"Partition ID":98,"Launch Time":1714515295912,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":93,"Index":92,"Attempt":0,"Partition ID":92,"Launch Time":1714515295908,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295912,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":921032,"Value":148983143,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":222,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":993900,"Value":195437754,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":232032,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":921032,"Executor Run Time":1,"Executor CPU Time":993900,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":95,"Index":94,"Attempt":0,"Partition ID":94,"Launch Time":1714515295909,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295913,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":624857,"Value":149608000,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":637098,"Value":196074852,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":234459,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":624857,"Executor Run Time":0,"Executor CPU Time":637098,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":100,"Index":99,"Attempt":0,"Partition ID":99,"Launch Time":1714515295913,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":97,"Index":96,"Attempt":0,"Partition ID":96,"Launch Time":1714515295911,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295914,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":655787,"Value":150263787,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":841095,"Value":196915947,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":236886,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":655787,"Executor Run Time":0,"Executor CPU Time":841095,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":101,"Index":100,"Attempt":0,"Partition ID":100,"Launch Time":1714515295914,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":102,"Index":101,"Attempt":0,"Partition ID":101,"Launch Time":1714515295914,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":96,"Index":95,"Attempt":0,"Partition ID":95,"Launch Time":1714515295910,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295914,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":114,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1347080,"Value":151610867,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":223,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1844045,"Value":198759992,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":239399,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1347080,"Executor Run Time":1,"Executor CPU Time":1844045,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":98,"Index":97,"Attempt":0,"Partition ID":97,"Launch Time":1714515295911,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295914,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":831147,"Value":152442014,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":224,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1014957,"Value":199774949,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":241869,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":831147,"Executor Run Time":1,"Executor CPU Time":1014957,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":103,"Index":102,"Attempt":0,"Partition ID":102,"Launch Time":1714515295916,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":100,"Index":99,"Attempt":0,"Partition ID":99,"Launch Time":1714515295913,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295916,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":779060,"Value":153221074,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":982780,"Value":200757729,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":244296,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":779060,"Executor Run Time":0,"Executor CPU Time":982780,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":104,"Index":103,"Attempt":0,"Partition ID":103,"Launch Time":1714515295916,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":99,"Index":98,"Attempt":0,"Partition ID":98,"Launch Time":1714515295912,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295916,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":115,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1641916,"Value":154862990,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":225,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1589408,"Value":202347137,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":246809,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1641916,"Executor Run Time":1,"Executor CPU Time":1589408,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":105,"Index":104,"Attempt":0,"Partition ID":104,"Launch Time":1714515295917,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":101,"Index":100,"Attempt":0,"Partition ID":100,"Launch Time":1714515295914,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295917,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":116,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1125091,"Value":155988081,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":998369,"Value":203345506,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":249279,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1125091,"Executor Run Time":0,"Executor CPU Time":998369,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":106,"Index":105,"Attempt":0,"Partition ID":105,"Launch Time":1714515295918,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":102,"Index":101,"Attempt":0,"Partition ID":101,"Launch Time":1714515295914,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295918,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":117,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1155310,"Value":157143391,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":226,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1487994,"Value":204833500,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":251792,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1155310,"Executor Run Time":1,"Executor CPU Time":1487994,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":107,"Index":106,"Attempt":0,"Partition ID":106,"Launch Time":1714515295919,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":103,"Index":102,"Attempt":0,"Partition ID":102,"Launch Time":1714515295916,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295919,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":959580,"Value":158102971,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":227,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":977000,"Value":205810500,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":254262,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":959580,"Executor Run Time":1,"Executor CPU Time":977000,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":108,"Index":107,"Attempt":0,"Partition ID":107,"Launch Time":1714515295920,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":105,"Index":104,"Attempt":0,"Partition ID":104,"Launch Time":1714515295917,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295921,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":118,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1023818,"Value":159126789,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":228,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1348520,"Value":207159020,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":256775,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1023818,"Executor Run Time":1,"Executor CPU Time":1348520,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":109,"Index":108,"Attempt":0,"Partition ID":108,"Launch Time":1714515295921,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":104,"Index":103,"Attempt":0,"Partition ID":103,"Launch Time":1714515295916,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295921,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":119,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1517912,"Value":160644701,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":229,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1841496,"Value":209000516,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":259288,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1517912,"Executor Run Time":1,"Executor CPU Time":1841496,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":110,"Index":109,"Attempt":0,"Partition ID":109,"Launch Time":1714515295922,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":107,"Index":106,"Attempt":0,"Partition ID":106,"Launch Time":1714515295919,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295923,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":120,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1185400,"Value":161830101,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":230,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1241816,"Value":210242332,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":261801,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1185400,"Executor Run Time":1,"Executor CPU Time":1241816,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":111,"Index":110,"Attempt":0,"Partition ID":110,"Launch Time":1714515295923,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":106,"Index":105,"Attempt":0,"Partition ID":105,"Launch Time":1714515295918,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295923,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":121,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1665054,"Value":163495155,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":231,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1424287,"Value":211666619,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":264314,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1665054,"Executor Run Time":1,"Executor CPU Time":1424287,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":112,"Index":111,"Attempt":0,"Partition ID":111,"Launch Time":1714515295923,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":108,"Index":107,"Attempt":0,"Partition ID":107,"Launch Time":1714515295920,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295924,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":696194,"Value":164191349,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":810510,"Value":212477129,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":266741,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":696194,"Executor Run Time":0,"Executor CPU Time":810510,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":113,"Index":112,"Attempt":0,"Partition ID":112,"Launch Time":1714515295924,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":109,"Index":108,"Attempt":0,"Partition ID":108,"Launch Time":1714515295921,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295924,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":855656,"Value":165047005,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":945281,"Value":213422410,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":269168,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":855656,"Executor Run Time":0,"Executor CPU Time":945281,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":114,"Index":113,"Attempt":0,"Partition ID":113,"Launch Time":1714515295926,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":110,"Index":109,"Attempt":0,"Partition ID":109,"Launch Time":1714515295922,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295926,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":946331,"Value":165993336,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":232,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1211986,"Value":214634396,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":271638,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":946331,"Executor Run Time":1,"Executor CPU Time":1211986,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":115,"Index":114,"Attempt":0,"Partition ID":114,"Launch Time":1714515295926,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":111,"Index":110,"Attempt":0,"Partition ID":110,"Launch Time":1714515295923,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295926,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":122,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1286653,"Value":167279989,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":981740,"Value":215616136,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":274108,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1286653,"Executor Run Time":0,"Executor CPU Time":981740,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":116,"Index":115,"Attempt":0,"Partition ID":115,"Launch Time":1714515295926,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":117,"Index":116,"Attempt":0,"Partition ID":116,"Launch Time":1714515295927,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":113,"Index":112,"Attempt":0,"Partition ID":112,"Launch Time":1714515295924,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295927,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":633297,"Value":167913286,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":894235,"Value":216510371,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":276535,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":633297,"Executor Run Time":0,"Executor CPU Time":894235,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":112,"Index":111,"Attempt":0,"Partition ID":111,"Launch Time":1714515295923,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295927,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":988528,"Value":168901814,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":233,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1101384,"Value":217611755,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":279005,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":988528,"Executor Run Time":1,"Executor CPU Time":1101384,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":118,"Index":117,"Attempt":0,"Partition ID":117,"Launch Time":1714515295929,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":119,"Index":118,"Attempt":0,"Partition ID":118,"Launch Time":1714515295929,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":114,"Index":113,"Attempt":0,"Partition ID":113,"Launch Time":1714515295926,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295929,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":123,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1284564,"Value":170186378,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":234,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1081114,"Value":218692869,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":281518,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1284564,"Executor Run Time":1,"Executor CPU Time":1081114,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":115,"Index":114,"Attempt":0,"Partition ID":114,"Launch Time":1714515295926,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295929,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":914533,"Value":171100911,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":235,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":977499,"Value":219670368,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":283988,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":914533,"Executor Run Time":1,"Executor CPU Time":977499,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":120,"Index":119,"Attempt":0,"Partition ID":119,"Launch Time":1714515295929,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":121,"Index":120,"Attempt":0,"Partition ID":120,"Launch Time":1714515295930,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":116,"Index":115,"Attempt":0,"Partition ID":115,"Launch Time":1714515295926,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295930,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":124,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1044916,"Value":172145827,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":944311,"Value":220614679,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":286458,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1044916,"Executor Run Time":0,"Executor CPU Time":944311,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":117,"Index":116,"Attempt":0,"Partition ID":116,"Launch Time":1714515295927,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295930,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":845338,"Value":172991165,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":236,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1091033,"Value":221705712,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":288928,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":845338,"Executor Run Time":1,"Executor CPU Time":1091033,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":122,"Index":121,"Attempt":0,"Partition ID":121,"Launch Time":1714515295932,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":118,"Index":117,"Attempt":0,"Partition ID":117,"Launch Time":1714515295929,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295932,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":125,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1304363,"Value":174295528,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":237,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1215747,"Value":222921459,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":291441,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1304363,"Executor Run Time":1,"Executor CPU Time":1215747,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":123,"Index":122,"Attempt":0,"Partition ID":122,"Launch Time":1714515295933,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":119,"Index":118,"Attempt":0,"Partition ID":118,"Launch Time":1714515295929,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295933,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":126,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1152170,"Value":175447698,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":238,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1012138,"Value":223933597,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":293954,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1152170,"Executor Run Time":1,"Executor CPU Time":1012138,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":124,"Index":123,"Attempt":0,"Partition ID":123,"Launch Time":1714515295933,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":125,"Index":124,"Attempt":0,"Partition ID":124,"Launch Time":1714515295934,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":120,"Index":119,"Attempt":0,"Partition ID":119,"Launch Time":1714515295929,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295934,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":127,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1555030,"Value":177002728,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":990739,"Value":224924336,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":296424,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1555030,"Executor Run Time":0,"Executor CPU Time":990739,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":121,"Index":120,"Attempt":0,"Partition ID":120,"Launch Time":1714515295930,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295934,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":128,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1319183,"Value":178321911,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":941162,"Value":225865498,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":298894,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1319183,"Executor Run Time":0,"Executor CPU Time":941162,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":126,"Index":125,"Attempt":0,"Partition ID":125,"Launch Time":1714515295935,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":123,"Index":122,"Attempt":0,"Partition ID":122,"Launch Time":1714515295933,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295935,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":612919,"Value":178934830,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":697903,"Value":226563401,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":301321,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":612919,"Executor Run Time":0,"Executor CPU Time":697903,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":127,"Index":126,"Attempt":0,"Partition ID":126,"Launch Time":1714515295936,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":122,"Index":121,"Attempt":0,"Partition ID":121,"Launch Time":1714515295932,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295936,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":781620,"Value":179716450,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":888214,"Value":227451615,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":303748,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":781620,"Executor Run Time":0,"Executor CPU Time":888214,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":128,"Index":127,"Attempt":0,"Partition ID":127,"Launch Time":1714515295938,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":124,"Index":123,"Attempt":0,"Partition ID":123,"Launch Time":1714515295933,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295938,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":129,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1055146,"Value":180771596,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":240,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1974428,"Value":229426043,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":306261,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1055146,"Executor Run Time":2,"Executor CPU Time":1974428,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":129,"Index":128,"Attempt":0,"Partition ID":128,"Launch Time":1714515295938,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":125,"Index":124,"Attempt":0,"Partition ID":124,"Launch Time":1714515295934,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295938,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":891585,"Value":181663181,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":241,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1506113,"Value":230932156,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":308731,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":891585,"Executor Run Time":1,"Executor CPU Time":1506113,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":130,"Index":129,"Attempt":0,"Partition ID":129,"Launch Time":1714515295939,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":126,"Index":125,"Attempt":0,"Partition ID":125,"Launch Time":1714515295935,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295939,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":944462,"Value":182607643,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":242,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1091884,"Value":232024040,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":311201,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":944462,"Executor Run Time":1,"Executor CPU Time":1091884,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":131,"Index":130,"Attempt":0,"Partition ID":130,"Launch Time":1714515295940,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":127,"Index":126,"Attempt":0,"Partition ID":126,"Launch Time":1714515295936,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295941,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":130,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1093553,"Value":183701196,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":244,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1387165,"Value":233411205,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":313714,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1093553,"Executor Run Time":2,"Executor CPU Time":1387165,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":132,"Index":131,"Attempt":0,"Partition ID":131,"Launch Time":1714515295941,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":128,"Index":127,"Attempt":0,"Partition ID":127,"Launch Time":1714515295938,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295941,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":748502,"Value":184449698,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":245,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1025823,"Value":234437028,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":316184,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":748502,"Executor Run Time":1,"Executor CPU Time":1025823,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":133,"Index":132,"Attempt":0,"Partition ID":132,"Launch Time":1714515295942,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":130,"Index":129,"Attempt":0,"Partition ID":129,"Launch Time":1714515295939,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295943,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":812778,"Value":185262476,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":246,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1533081,"Value":235970109,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":318654,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":812778,"Executor Run Time":1,"Executor CPU Time":1533081,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":134,"Index":133,"Attempt":0,"Partition ID":133,"Launch Time":1714515295943,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":129,"Index":128,"Attempt":0,"Partition ID":128,"Launch Time":1714515295938,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295943,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":131,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1333271,"Value":186595747,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":247,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1579598,"Value":237549707,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":321167,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1333271,"Executor Run Time":1,"Executor CPU Time":1579598,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":135,"Index":134,"Attempt":0,"Partition ID":134,"Launch Time":1714515295943,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":131,"Index":130,"Attempt":0,"Partition ID":130,"Launch Time":1714515295940,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295943,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":797249,"Value":187392996,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":898444,"Value":238448151,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":323594,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":797249,"Executor Run Time":0,"Executor CPU Time":898444,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":136,"Index":135,"Attempt":0,"Partition ID":135,"Launch Time":1714515295944,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":132,"Index":131,"Attempt":0,"Partition ID":131,"Launch Time":1714515295941,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295944,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":132,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":896482,"Value":188289478,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":857677,"Value":239305828,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":326064,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":896482,"Executor Run Time":0,"Executor CPU Time":857677,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":137,"Index":136,"Attempt":0,"Partition ID":136,"Launch Time":1714515295946,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":134,"Index":133,"Attempt":0,"Partition ID":133,"Launch Time":1714515295943,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295946,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":133,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1376369,"Value":189665847,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":248,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1003749,"Value":240309577,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":328577,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1376369,"Executor Run Time":1,"Executor CPU Time":1003749,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":138,"Index":137,"Attempt":0,"Partition ID":137,"Launch Time":1714515295946,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":136,"Index":135,"Attempt":0,"Partition ID":135,"Launch Time":1714515295944,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295947,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":792360,"Value":190458207,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":834937,"Value":241144514,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":331004,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":792360,"Executor Run Time":0,"Executor CPU Time":834937,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":139,"Index":138,"Attempt":0,"Partition ID":138,"Launch Time":1714515295947,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":133,"Index":132,"Attempt":0,"Partition ID":132,"Launch Time":1714515295942,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295947,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":134,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1806757,"Value":192264964,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":249,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1420087,"Value":242564601,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":333517,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1806757,"Executor Run Time":1,"Executor CPU Time":1420087,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":140,"Index":139,"Attempt":0,"Partition ID":139,"Launch Time":1714515295947,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":135,"Index":134,"Attempt":0,"Partition ID":134,"Launch Time":1714515295943,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295947,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":135,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1295793,"Value":193560757,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":250,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1221328,"Value":243785929,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":336030,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1295793,"Executor Run Time":1,"Executor CPU Time":1221328,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":141,"Index":140,"Attempt":0,"Partition ID":140,"Launch Time":1714515295949,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":137,"Index":136,"Attempt":0,"Partition ID":136,"Launch Time":1714515295946,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295949,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":136,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1187809,"Value":194748566,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":819648,"Value":244605577,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":338500,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1187809,"Executor Run Time":0,"Executor CPU Time":819648,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":142,"Index":141,"Attempt":0,"Partition ID":141,"Launch Time":1714515295950,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":140,"Index":139,"Attempt":0,"Partition ID":139,"Launch Time":1714515295947,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295950,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":137,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1221537,"Value":195970103,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":933443,"Value":245539020,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":340970,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1221537,"Executor Run Time":0,"Executor CPU Time":933443,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":143,"Index":142,"Attempt":0,"Partition ID":142,"Launch Time":1714515295950,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":138,"Index":137,"Attempt":0,"Partition ID":137,"Launch Time":1714515295946,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295951,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":138,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1488133,"Value":197458236,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":251,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1520802,"Value":247059822,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":343483,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1488133,"Executor Run Time":1,"Executor CPU Time":1520802,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":144,"Index":143,"Attempt":0,"Partition ID":143,"Launch Time":1714515295951,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":139,"Index":138,"Attempt":0,"Partition ID":138,"Launch Time":1714515295947,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295951,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":139,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1480294,"Value":198938530,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":252,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1196788,"Value":248256610,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":345996,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1480294,"Executor Run Time":1,"Executor CPU Time":1196788,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":145,"Index":144,"Attempt":0,"Partition ID":144,"Launch Time":1714515295951,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":141,"Index":140,"Attempt":0,"Partition ID":140,"Launch Time":1714515295949,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295952,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":584870,"Value":199523400,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":715443,"Value":248972053,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":348423,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":584870,"Executor Run Time":0,"Executor CPU Time":715443,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":146,"Index":145,"Attempt":0,"Partition ID":145,"Launch Time":1714515295954,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":144,"Index":143,"Attempt":0,"Partition ID":143,"Launch Time":1714515295951,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295954,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":140,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1022568,"Value":200545968,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":253,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1222617,"Value":250194670,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":350936,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1022568,"Executor Run Time":1,"Executor CPU Time":1222617,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":147,"Index":146,"Attempt":0,"Partition ID":146,"Launch Time":1714515295955,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":148,"Index":147,"Attempt":0,"Partition ID":147,"Launch Time":1714515295955,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":145,"Index":144,"Attempt":0,"Partition ID":144,"Launch Time":1714515295951,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295955,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":793200,"Value":201339168,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":937851,"Value":251132521,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":353363,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":793200,"Executor Run Time":0,"Executor CPU Time":937851,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":149,"Index":148,"Attempt":0,"Partition ID":148,"Launch Time":1714515295955,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":143,"Index":142,"Attempt":0,"Partition ID":142,"Launch Time":1714515295950,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295955,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":141,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1525861,"Value":202865029,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":254,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1128292,"Value":252260813,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":355876,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1525861,"Executor Run Time":1,"Executor CPU Time":1128292,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":142,"Index":141,"Attempt":0,"Partition ID":141,"Launch Time":1714515295950,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295955,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":142,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1315772,"Value":204180801,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":255,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1731091,"Value":253991904,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":358389,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1315772,"Executor Run Time":1,"Executor CPU Time":1731091,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":150,"Index":149,"Attempt":0,"Partition ID":149,"Launch Time":1714515295958,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":147,"Index":146,"Attempt":0,"Partition ID":146,"Launch Time":1714515295955,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295958,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":143,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1309152,"Value":205489953,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":256,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1149900,"Value":255141804,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":360902,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1309152,"Executor Run Time":1,"Executor CPU Time":1149900,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":151,"Index":150,"Attempt":0,"Partition ID":150,"Launch Time":1714515295958,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":152,"Index":151,"Attempt":0,"Partition ID":151,"Launch Time":1714515295959,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":146,"Index":145,"Attempt":0,"Partition ID":145,"Launch Time":1714515295954,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295959,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":144,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1500233,"Value":206990186,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":257,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1296913,"Value":256438717,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":363415,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1500233,"Executor Run Time":1,"Executor CPU Time":1296913,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":149,"Index":148,"Attempt":0,"Partition ID":148,"Launch Time":1714515295955,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295959,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":885924,"Value":207876110,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":258,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1226277,"Value":257664994,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":365885,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":885924,"Executor Run Time":1,"Executor CPU Time":1226277,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":153,"Index":152,"Attempt":0,"Partition ID":152,"Launch Time":1714515295959,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":148,"Index":147,"Attempt":0,"Partition ID":147,"Launch Time":1714515295955,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295959,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":145,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1158020,"Value":209034130,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":259,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1264855,"Value":258929849,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":368398,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1158020,"Executor Run Time":1,"Executor CPU Time":1264855,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":154,"Index":153,"Attempt":0,"Partition ID":153,"Launch Time":1714515295960,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":150,"Index":149,"Attempt":0,"Partition ID":149,"Launch Time":1714515295958,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295961,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":916113,"Value":209950243,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":964090,"Value":259893939,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":370825,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":916113,"Executor Run Time":0,"Executor CPU Time":964090,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":155,"Index":154,"Attempt":0,"Partition ID":154,"Launch Time":1714515295961,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":151,"Index":150,"Attempt":0,"Partition ID":150,"Launch Time":1714515295958,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295961,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":146,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1043636,"Value":210993879,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":684935,"Value":260578874,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":373295,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1043636,"Executor Run Time":0,"Executor CPU Time":684935,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":156,"Index":155,"Attempt":0,"Partition ID":155,"Launch Time":1714515295962,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":152,"Index":151,"Attempt":0,"Partition ID":151,"Launch Time":1714515295959,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295962,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":147,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1116132,"Value":212110011,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":260,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1365000,"Value":261943874,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":375808,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1116132,"Executor Run Time":1,"Executor CPU Time":1365000,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":157,"Index":156,"Attempt":0,"Partition ID":156,"Launch Time":1714515295962,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":153,"Index":152,"Attempt":0,"Partition ID":152,"Launch Time":1714515295959,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295963,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":852156,"Value":212962167,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":261,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1258055,"Value":263201929,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":378278,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":852156,"Executor Run Time":1,"Executor CPU Time":1258055,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":158,"Index":157,"Attempt":0,"Partition ID":157,"Launch Time":1714515295963,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":155,"Index":154,"Attempt":0,"Partition ID":154,"Launch Time":1714515295961,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295963,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":561102,"Value":213523269,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":622128,"Value":263824057,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":380705,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":561102,"Executor Run Time":0,"Executor CPU Time":622128,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":159,"Index":158,"Attempt":0,"Partition ID":158,"Launch Time":1714515295963,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":154,"Index":153,"Attempt":0,"Partition ID":153,"Launch Time":1714515295960,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295964,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":718254,"Value":214241523,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":789569,"Value":264613626,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":383132,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":718254,"Executor Run Time":0,"Executor CPU Time":789569,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":160,"Index":159,"Attempt":0,"Partition ID":159,"Launch Time":1714515295965,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":161,"Index":160,"Attempt":0,"Partition ID":160,"Launch Time":1714515295965,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":157,"Index":156,"Attempt":0,"Partition ID":156,"Launch Time":1714515295962,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295965,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":962540,"Value":215204063,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":897334,"Value":265510960,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":385559,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":962540,"Executor Run Time":0,"Executor CPU Time":897334,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":158,"Index":157,"Attempt":0,"Partition ID":157,"Launch Time":1714515295963,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295965,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":788339,"Value":215992402,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":890894,"Value":266401854,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":387986,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":788339,"Executor Run Time":0,"Executor CPU Time":890894,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":162,"Index":161,"Attempt":0,"Partition ID":161,"Launch Time":1714515295966,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":156,"Index":155,"Attempt":0,"Partition ID":155,"Launch Time":1714515295962,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295966,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":148,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1565119,"Value":217557521,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":262,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1210968,"Value":267612822,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":390499,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1565119,"Executor Run Time":1,"Executor CPU Time":1210968,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":163,"Index":162,"Attempt":0,"Partition ID":162,"Launch Time":1714515295966,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":159,"Index":158,"Attempt":0,"Partition ID":158,"Launch Time":1714515295963,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295966,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":944511,"Value":218502032,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":263,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1037957,"Value":268650779,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":392969,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":944511,"Executor Run Time":1,"Executor CPU Time":1037957,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":164,"Index":163,"Attempt":0,"Partition ID":163,"Launch Time":1714515295967,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":160,"Index":159,"Attempt":0,"Partition ID":159,"Launch Time":1714515295965,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295967,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":791809,"Value":219293841,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":774390,"Value":269425169,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":395396,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":791809,"Executor Run Time":0,"Executor CPU Time":774390,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":165,"Index":164,"Attempt":0,"Partition ID":164,"Launch Time":1714515295968,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":161,"Index":160,"Attempt":0,"Partition ID":160,"Launch Time":1714515295965,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295968,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":149,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1083523,"Value":220377364,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":742373,"Value":270167542,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":397866,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1083523,"Executor Run Time":0,"Executor CPU Time":742373,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":166,"Index":165,"Attempt":0,"Partition ID":165,"Launch Time":1714515295969,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":167,"Index":166,"Attempt":0,"Partition ID":166,"Launch Time":1714515295969,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":162,"Index":161,"Attempt":0,"Partition ID":161,"Launch Time":1714515295966,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295969,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":150,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1072184,"Value":221449548,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":264,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1525922,"Value":271693464,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":400379,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1072184,"Executor Run Time":1,"Executor CPU Time":1525922,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":163,"Index":162,"Attempt":0,"Partition ID":162,"Launch Time":1714515295966,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295970,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":151,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1005609,"Value":222455157,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":265,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1061454,"Value":272754918,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":402892,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1005609,"Executor Run Time":1,"Executor CPU Time":1061454,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":168,"Index":167,"Attempt":0,"Partition ID":167,"Launch Time":1714515295975,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":169,"Index":168,"Attempt":0,"Partition ID":168,"Launch Time":1714515295975,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":165,"Index":164,"Attempt":0,"Partition ID":164,"Launch Time":1714515295968,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295975,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":742231,"Value":223197388,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":266,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1097324,"Value":273852242,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":405362,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":742231,"Executor Run Time":1,"Executor CPU Time":1097324,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":164,"Index":163,"Attempt":0,"Partition ID":163,"Launch Time":1714515295967,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295976,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":864605,"Value":224061993,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":267,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1642096,"Value":275494338,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":407832,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":864605,"Executor Run Time":1,"Executor CPU Time":1642096,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":170,"Index":169,"Attempt":0,"Partition ID":169,"Launch Time":1714515295977,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":166,"Index":165,"Attempt":0,"Partition ID":165,"Launch Time":1714515295969,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295977,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":156,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1148711,"Value":225210704,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":268,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1846195,"Value":277340533,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2556,"Value":410388,"Internal":true,"Count Failed Values":true},{"ID":45,"Name":"internal.metrics.jvmGCTime","Update":4,"Value":10,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":1148711,"Executor Run Time":1,"Executor CPU Time":1846195,"Peak Execution Memory":0,"Result Size":2556,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":171,"Index":170,"Attempt":0,"Partition ID":170,"Launch Time":1714515295978,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":167,"Index":166,"Attempt":0,"Partition ID":166,"Launch Time":1714515295969,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295978,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":160,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":957779,"Value":226168483,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":269,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1709283,"Value":279049816,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2556,"Value":412944,"Internal":true,"Count Failed Values":true},{"ID":45,"Name":"internal.metrics.jvmGCTime","Update":4,"Value":14,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":957779,"Executor Run Time":1,"Executor CPU Time":1709283,"Peak Execution Memory":0,"Result Size":2556,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":172,"Index":171,"Attempt":0,"Partition ID":171,"Launch Time":1714515295978,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":173,"Index":172,"Attempt":0,"Partition ID":172,"Launch Time":1714515295979,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":169,"Index":168,"Attempt":0,"Partition ID":168,"Launch Time":1714515295975,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295979,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":989079,"Value":227157562,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":270,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1125362,"Value":280175178,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":415414,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":989079,"Executor Run Time":1,"Executor CPU Time":1125362,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":168,"Index":167,"Attempt":0,"Partition ID":167,"Launch Time":1714515295975,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295979,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":161,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1279904,"Value":228437466,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":271,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1235716,"Value":281410894,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":417927,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1279904,"Executor Run Time":1,"Executor CPU Time":1235716,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":174,"Index":173,"Attempt":0,"Partition ID":173,"Launch Time":1714515295980,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":170,"Index":169,"Attempt":0,"Partition ID":169,"Launch Time":1714515295977,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295981,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":162,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1145771,"Value":229583237,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":272,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1145971,"Value":282556865,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":420440,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1145771,"Executor Run Time":1,"Executor CPU Time":1145971,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":175,"Index":174,"Attempt":0,"Partition ID":174,"Launch Time":1714515295981,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":171,"Index":170,"Attempt":0,"Partition ID":170,"Launch Time":1714515295978,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295981,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":163,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1120913,"Value":230704150,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":273,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1075664,"Value":283632529,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":422953,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1120913,"Executor Run Time":1,"Executor CPU Time":1075664,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":176,"Index":175,"Attempt":0,"Partition ID":175,"Launch Time":1714515295981,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":177,"Index":176,"Attempt":0,"Partition ID":176,"Launch Time":1714515295981,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":173,"Index":172,"Attempt":0,"Partition ID":172,"Launch Time":1714515295979,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295982,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":684986,"Value":231389136,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":834207,"Value":284466736,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":425380,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":684986,"Executor Run Time":0,"Executor CPU Time":834207,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":172,"Index":171,"Attempt":0,"Partition ID":171,"Launch Time":1714515295978,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295982,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":809189,"Value":232198325,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":274,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1049446,"Value":285516182,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":427850,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":809189,"Executor Run Time":1,"Executor CPU Time":1049446,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":178,"Index":177,"Attempt":0,"Partition ID":177,"Launch Time":1714515295983,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":175,"Index":174,"Attempt":0,"Partition ID":174,"Launch Time":1714515295981,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295984,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":944842,"Value":233143167,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":890424,"Value":286406606,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":430277,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":944842,"Executor Run Time":0,"Executor CPU Time":890424,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":179,"Index":178,"Attempt":0,"Partition ID":178,"Launch Time":1714515295984,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":174,"Index":173,"Attempt":0,"Partition ID":173,"Launch Time":1714515295980,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295984,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":164,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1143651,"Value":234286818,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":275,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1004568,"Value":287411174,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":432790,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1143651,"Executor Run Time":1,"Executor CPU Time":1004568,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":180,"Index":179,"Attempt":0,"Partition ID":179,"Launch Time":1714515295984,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":177,"Index":176,"Attempt":0,"Partition ID":176,"Launch Time":1714515295981,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295984,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":665295,"Value":234952113,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":761561,"Value":288172735,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":435217,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":665295,"Executor Run Time":0,"Executor CPU Time":761561,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":181,"Index":180,"Attempt":0,"Partition ID":180,"Launch Time":1714515295984,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":176,"Index":175,"Attempt":0,"Partition ID":175,"Launch Time":1714515295981,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295985,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":772510,"Value":235724623,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":949951,"Value":289122686,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":437644,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":772510,"Executor Run Time":0,"Executor CPU Time":949951,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":182,"Index":181,"Attempt":0,"Partition ID":181,"Launch Time":1714515295986,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":178,"Index":177,"Attempt":0,"Partition ID":177,"Launch Time":1714515295983,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295986,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":694324,"Value":236418947,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":810889,"Value":289933575,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":440071,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":694324,"Executor Run Time":0,"Executor CPU Time":810889,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":183,"Index":182,"Attempt":0,"Partition ID":182,"Launch Time":1714515295987,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":179,"Index":178,"Attempt":0,"Partition ID":178,"Launch Time":1714515295984,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295987,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":754861,"Value":237173808,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":276,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1072725,"Value":291006300,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":442541,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":754861,"Executor Run Time":1,"Executor CPU Time":1072725,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":184,"Index":183,"Attempt":0,"Partition ID":183,"Launch Time":1714515295988,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":180,"Index":179,"Attempt":0,"Partition ID":179,"Launch Time":1714515295984,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295988,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":165,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1484004,"Value":238657812,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":277,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1681163,"Value":292687463,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":445054,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1484004,"Executor Run Time":1,"Executor CPU Time":1681163,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":185,"Index":184,"Attempt":0,"Partition ID":184,"Launch Time":1714515295989,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":181,"Index":180,"Attempt":0,"Partition ID":180,"Launch Time":1714515295984,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295989,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":166,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":977730,"Value":239635542,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":278,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1598678,"Value":294286141,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":447567,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":977730,"Executor Run Time":1,"Executor CPU Time":1598678,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":186,"Index":185,"Attempt":0,"Partition ID":185,"Launch Time":1714515295989,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":182,"Index":181,"Attempt":0,"Partition ID":181,"Launch Time":1714515295986,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295989,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":766551,"Value":240402093,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":818617,"Value":295104758,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":449994,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":766551,"Executor Run Time":0,"Executor CPU Time":818617,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":187,"Index":186,"Attempt":0,"Partition ID":186,"Launch Time":1714515295989,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":183,"Index":182,"Attempt":0,"Partition ID":182,"Launch Time":1714515295987,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295990,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":666226,"Value":241068319,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":812248,"Value":295917006,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":452421,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":666226,"Executor Run Time":0,"Executor CPU Time":812248,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":188,"Index":187,"Attempt":0,"Partition ID":187,"Launch Time":1714515295991,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":184,"Index":183,"Attempt":0,"Partition ID":183,"Launch Time":1714515295988,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295991,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":673115,"Value":241741434,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":871445,"Value":296788451,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":454848,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":673115,"Executor Run Time":0,"Executor CPU Time":871445,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":189,"Index":188,"Attempt":0,"Partition ID":188,"Launch Time":1714515295991,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":185,"Index":184,"Attempt":0,"Partition ID":184,"Launch Time":1714515295989,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295991,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":167,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1055826,"Value":242797260,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":739282,"Value":297527733,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":457318,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1055826,"Executor Run Time":0,"Executor CPU Time":739282,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":190,"Index":189,"Attempt":0,"Partition ID":189,"Launch Time":1714515295993,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":187,"Index":186,"Attempt":0,"Partition ID":186,"Launch Time":1714515295989,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295993,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":935242,"Value":243732502,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":279,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1319621,"Value":298847354,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":459788,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":935242,"Executor Run Time":1,"Executor CPU Time":1319621,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":191,"Index":190,"Attempt":0,"Partition ID":190,"Launch Time":1714515295993,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":186,"Index":185,"Attempt":0,"Partition ID":185,"Launch Time":1714515295989,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295993,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1093274,"Value":244825776,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":280,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1569239,"Value":300416593,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":462301,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1093274,"Executor Run Time":1,"Executor CPU Time":1569239,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":192,"Index":191,"Attempt":0,"Partition ID":191,"Launch Time":1714515295993,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":193,"Index":192,"Attempt":0,"Partition ID":192,"Launch Time":1714515295994,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":189,"Index":188,"Attempt":0,"Partition ID":188,"Launch Time":1714515295991,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295994,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":589209,"Value":245414985,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":708435,"Value":301125028,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":464728,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":589209,"Executor Run Time":0,"Executor CPU Time":708435,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":188,"Index":187,"Attempt":0,"Partition ID":187,"Launch Time":1714515295991,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295994,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":591840,"Value":246006825,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":724452,"Value":301849480,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":467155,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":591840,"Executor Run Time":0,"Executor CPU Time":724452,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":194,"Index":193,"Attempt":0,"Partition ID":193,"Launch Time":1714515295995,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":190,"Index":189,"Attempt":0,"Partition ID":189,"Launch Time":1714515295993,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295996,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":824758,"Value":246831583,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":911943,"Value":302761423,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":469582,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":824758,"Executor Run Time":0,"Executor CPU Time":911943,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":195,"Index":194,"Attempt":0,"Partition ID":194,"Launch Time":1714515295996,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":192,"Index":191,"Attempt":0,"Partition ID":191,"Launch Time":1714515295993,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295996,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":788000,"Value":247619583,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":872315,"Value":303633738,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":472009,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":788000,"Executor Run Time":0,"Executor CPU Time":872315,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":196,"Index":195,"Attempt":0,"Partition ID":195,"Launch Time":1714515295997,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":191,"Index":190,"Attempt":0,"Partition ID":190,"Launch Time":1714515295993,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295997,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":169,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1392458,"Value":249012041,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":281,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1756820,"Value":305390558,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":474522,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1392458,"Executor Run Time":1,"Executor CPU Time":1756820,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":197,"Index":196,"Attempt":0,"Partition ID":196,"Launch Time":1714515295998,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":193,"Index":192,"Attempt":0,"Partition ID":192,"Launch Time":1714515295994,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295998,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":170,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1120792,"Value":250132833,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":282,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1062555,"Value":306453113,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":477035,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1120792,"Executor Run Time":1,"Executor CPU Time":1062555,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":198,"Index":197,"Attempt":0,"Partition ID":197,"Launch Time":1714515295998,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":199,"Index":198,"Attempt":0,"Partition ID":198,"Launch Time":1714515295999,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":194,"Index":193,"Attempt":0,"Partition ID":193,"Launch Time":1714515295995,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295999,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":730732,"Value":250863565,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":668806,"Value":307121919,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":479462,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":730732,"Executor Run Time":0,"Executor CPU Time":668806,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":195,"Index":194,"Attempt":0,"Partition ID":194,"Launch Time":1714515295996,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515295999,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":607279,"Value":251470844,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":610498,"Value":307732417,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":481889,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":607279,"Executor Run Time":0,"Executor CPU Time":610498,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":200,"Index":199,"Attempt":0,"Partition ID":199,"Launch Time":1714515296000,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":197,"Index":196,"Attempt":0,"Partition ID":196,"Launch Time":1714515295998,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296001,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":804748,"Value":252275592,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":917973,"Value":308650390,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":484316,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":804748,"Executor Run Time":0,"Executor CPU Time":917973,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":196,"Index":195,"Attempt":0,"Partition ID":195,"Launch Time":1714515295997,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296002,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":946171,"Value":253221763,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":283,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1141902,"Value":309792292,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":486786,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":946171,"Executor Run Time":1,"Executor CPU Time":1141902,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":198,"Index":197,"Attempt":0,"Partition ID":197,"Launch Time":1714515295998,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296002,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":834346,"Value":254056109,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":644027,"Value":310436319,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":489213,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":834346,"Executor Run Time":0,"Executor CPU Time":644027,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":199,"Index":198,"Attempt":0,"Partition ID":198,"Launch Time":1714515295999,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296002,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":897993,"Value":254954102,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":963651,"Value":311399970,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":491640,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":897993,"Executor Run Time":0,"Executor CPU Time":963651,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":200,"Index":199,"Attempt":0,"Partition ID":199,"Launch Time":1714515296000,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296003,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":660505,"Value":255614607,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":284,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1035719,"Value":312435689,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":494110,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":660505,"Executor Run Time":1,"Executor CPU Time":1035719,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":68,"Attempt":0,"Partition ID":68,"Launch Time":1714515295757,"Executor ID":"driver","Host":"10.69.200.180","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296272,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"local blocks read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"local bytes read","Update":"95","Value":"95","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":9,"Name":"fetch wait time","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":10,"Name":"records read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":9,"Value":179,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8567791,"Value":264182398,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":494,"Value":778,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":454212465,"Value":766648154,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":5305,"Value":499415,"Internal":true,"Count Failed Values":true},{"ID":45,"Name":"internal.metrics.jvmGCTime","Update":10,"Value":24,"Internal":true,"Count Failed Values":true},{"ID":46,"Name":"internal.metrics.resultSerializationTime","Update":3,"Value":7,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":95,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.output.bytesWritten","Update":932,"Value":932,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.output.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":9,"Executor Deserialize CPU Time":8567791,"Executor Run Time":494,"Executor CPU Time":454212465,"Peak Execution Memory":0,"Result Size":5305,"JVM GC Time":10,"Result Serialization Time":3,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":95,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":932,"Records Written":1},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"processLine at CliDriver.java:336","Number of Tasks":200,"RDD Info":[{"RDD ID":3,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"4\",\"name\":\"Exchange\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"UNORDERED","Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[1],"Details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1714515295739,"Completion Time":1714515296273,"Accumulables":[{"ID":5,"Name":"local blocks read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"local bytes read","Value":"95","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":9,"Name":"fetch wait time","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":10,"Name":"records read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Value":179,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Value":264182398,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Value":778,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Value":766648154,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Value":499415,"Internal":true,"Count Failed Values":true},{"ID":45,"Name":"internal.metrics.jvmGCTime","Value":24,"Internal":true,"Count Failed Values":true},{"ID":46,"Name":"internal.metrics.resultSerializationTime","Value":7,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.output.bytesWritten","Value":932,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.output.recordsWritten","Value":1,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":1,"Completion Time":1714515296275,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":4,"time":1714515296473} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":5,"description":"\nINSERT INTO local.developers VALUES(\"krisnova\", \"Kris\", \"Nova\")","details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nCommandResult (1)\n +- AppendData (6)\n +- AdaptiveSparkPlan (5)\n +- == Final Plan ==\n ShuffleQueryStage (4), Statistics(sizeInBytes=56.0 B, rowCount=1)\n +- Exchange (3)\n +- LocalTableScan (2)\n +- == Initial Plan ==\n Exchange (3)\n +- LocalTableScan (2)\n\n\n(1) CommandResult\nOutput: []\nArguments: \n\n(2) LocalTableScan\nOutput [3]: [username#6, firstname#7, lastname#8]\nArguments: [username#6, firstname#7, lastname#8]\n\n(3) Exchange\nInput [3]: [username#6, firstname#7, lastname#8]\nArguments: hashpartitioning(username#6, 200), REPARTITION_BY_NUM, [plan_id=21]\n\n(4) ShuffleQueryStage\nOutput [3]: [username#6, firstname#7, lastname#8]\nArguments: 0\n\n(5) AdaptiveSparkPlan\nOutput [3]: [username#6, firstname#7, lastname#8]\nArguments: isFinalPlan=true\n\n(6) AppendData\nInput [3]: [username#6, firstname#7, lastname#8]\nArguments: org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$2253/0x0000775334f5b4c0@205c5cd0, IcebergWrite(table=local.developers, format=PARQUET)\n\n","sparkPlanInfo":{"nodeName":"CommandResult","simpleString":"CommandResult ","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":65,"metricType":"sum"}]},"time":1714515296479,"modifiedConfigs":{}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":5,"time":1714515296480} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":6,"description":"\nINSERT INTO local.projects VALUES(\"krisnova\", \"aurae\")","details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nAppendData (4)\n+- AdaptiveSparkPlan (3)\n +- Exchange (2)\n +- LocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [2]: [creator#13, projectname#14]\nArguments: [creator#13, projectname#14]\n\n(2) Exchange\nInput [2]: [creator#13, projectname#14]\nArguments: hashpartitioning(creator#13, 200), REPARTITION_BY_NUM, [plan_id=42]\n\n(3) AdaptiveSparkPlan\nOutput [2]: [creator#13, projectname#14]\nArguments: isFinalPlan=false\n\n(4) AppendData\nInput [2]: [creator#13, projectname#14]\nArguments: org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$2253/0x0000775334f5b4c0@72e89ce2, IcebergWrite(table=local.projects, format=PARQUET)\n\n","sparkPlanInfo":{"nodeName":"AppendData","simpleString":"AppendData org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$2253/0x0000775334f5b4c0@72e89ce2, IcebergWrite(table=local.projects, format=PARQUET)","children":[{"nodeName":"AdaptiveSparkPlan","simpleString":"AdaptiveSparkPlan isFinalPlan=false","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(creator#13, 200), REPARTITION_BY_NUM, [plan_id=42]","children":[{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [creator#13, projectname#14]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":78,"metricType":"sum"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":76,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":77,"metricType":"nsTiming"},{"name":"records read","accumulatorId":74,"metricType":"sum"},{"name":"local bytes read","accumulatorId":72,"metricType":"size"},{"name":"fetch wait time","accumulatorId":73,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":70,"metricType":"size"},{"name":"local blocks read","accumulatorId":69,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":68,"metricType":"sum"},{"name":"data size","accumulatorId":66,"metricType":"size"},{"name":"number of partitions","accumulatorId":67,"metricType":"sum"},{"name":"remote bytes read to disk","accumulatorId":71,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":75,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]},"time":1714515296496,"modifiedConfigs":{}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate","executionId":6,"physicalPlanDescription":"== Physical Plan ==\nAppendData (5)\n+- AdaptiveSparkPlan (4)\n +- == Current Plan ==\n ShuffleQueryStage (3)\n +- Exchange (2)\n +- LocalTableScan (1)\n +- == Initial Plan ==\n Exchange (2)\n +- LocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [2]: [creator#13, projectname#14]\nArguments: [creator#13, projectname#14]\n\n(2) Exchange\nInput [2]: [creator#13, projectname#14]\nArguments: hashpartitioning(creator#13, 200), REPARTITION_BY_NUM, [plan_id=42]\n\n(3) ShuffleQueryStage\nOutput [2]: [creator#13, projectname#14]\nArguments: 0\n\n(4) AdaptiveSparkPlan\nOutput [2]: [creator#13, projectname#14]\nArguments: isFinalPlan=false\n\n(5) AppendData\nInput [2]: [creator#13, projectname#14]\nArguments: org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$2253/0x0000775334f5b4c0@72e89ce2, IcebergWrite(table=local.projects, format=PARQUET)\n\n","sparkPlanInfo":{"nodeName":"AppendData","simpleString":"AppendData org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$2253/0x0000775334f5b4c0@72e89ce2, IcebergWrite(table=local.projects, format=PARQUET)","children":[{"nodeName":"AdaptiveSparkPlan","simpleString":"AdaptiveSparkPlan isFinalPlan=false","children":[{"nodeName":"ShuffleQueryStage","simpleString":"ShuffleQueryStage 0","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(creator#13, 200), REPARTITION_BY_NUM, [plan_id=42]","children":[{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [creator#13, projectname#14]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":78,"metricType":"sum"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":76,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":77,"metricType":"nsTiming"},{"name":"records read","accumulatorId":74,"metricType":"sum"},{"name":"local bytes read","accumulatorId":72,"metricType":"size"},{"name":"fetch wait time","accumulatorId":73,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":70,"metricType":"size"},{"name":"local blocks read","accumulatorId":69,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":68,"metricType":"sum"},{"name":"data size","accumulatorId":66,"metricType":"size"},{"name":"number of partitions","accumulatorId":67,"metricType":"sum"},{"name":"remote bytes read to disk","accumulatorId":71,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":75,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":6,"accumUpdates":[[67,200]]} +{"Event":"SparkListenerJobStart","Job ID":2,"Submission Time":1714515296509,"Stage Infos":[{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"processLine at CliDriver.java:336","Number of Tasks":1,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"Exchange\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"LocalTableScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"11\",\"name\":\"LocalTableScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[3],"Properties":{"spark.sql.warehouse.dir":"file:/home/holden/repos/sparklens/spark-warehouse","spark.executor.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.driver.host":"10.69.200.180","spark.eventLog.enabled":"true","spark.driver.port":"37317","__fetch_continuous_blocks_in_batch_enabled":"true","spark.sql.catalog.local.type":"hadoop","spark.jars":"","spark.sql.catalog.spark_catalog":"org.apache.iceberg.spark.SparkSessionCatalog","spark.sql.datetime.java8API.enabled":"true","spark.sql.hive.version":"2.3.9","spark.app.name":"fresh","spark.rdd.scope":"{\"id\":\"10\",\"name\":\"Exchange\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.submitTime":"1714515291527","spark.job.description":"\nINSERT INTO local.projects VALUES(\"krisnova\", \"aurae\")","spark.app.startTime":"1714515291883","spark.executor.id":"driver","spark.sql.catalog.local.warehouse":"/home/holden/repos/sparklens/warehouse","spark.driver.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.sql.catalog.local":"org.apache.iceberg.spark.SparkCatalog","spark.submit.deployMode":"client","spark.master":"local[5]","spark.sql.execution.id":"6","spark.sql.catalogImplementation":"hive","spark.sql.catalog.spark_catalog.type":"hive","spark.app.id":"local-1714515292260"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"processLine at CliDriver.java:336","Number of Tasks":1,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"Exchange\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"LocalTableScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"11\",\"name\":\"LocalTableScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1714515296510,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/holden/repos/sparklens/spark-warehouse","spark.executor.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.driver.host":"10.69.200.180","spark.eventLog.enabled":"true","spark.driver.port":"37317","__fetch_continuous_blocks_in_batch_enabled":"true","spark.sql.catalog.local.type":"hadoop","spark.jars":"","spark.sql.catalog.spark_catalog":"org.apache.iceberg.spark.SparkSessionCatalog","spark.sql.datetime.java8API.enabled":"true","spark.sql.hive.version":"2.3.9","spark.app.name":"fresh","spark.rdd.scope":"{\"id\":\"10\",\"name\":\"Exchange\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.submitTime":"1714515291527","spark.job.description":"\nINSERT INTO local.projects VALUES(\"krisnova\", \"aurae\")","spark.app.startTime":"1714515291883","spark.executor.id":"driver","spark.sql.catalog.local.warehouse":"/home/holden/repos/sparklens/warehouse","spark.driver.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.sql.catalog.local":"org.apache.iceberg.spark.SparkCatalog","spark.submit.deployMode":"client","spark.master":"local[5]","spark.sql.execution.id":"6","spark.sql.catalogImplementation":"hive","spark.sql.catalog.spark_catalog.type":"hive","spark.app.id":"local-1714515292260"}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":201,"Index":0,"Attempt":0,"Partition ID":0,"Launch Time":1714515296512,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":201,"Index":0,"Attempt":0,"Partition ID":0,"Launch Time":1714515296512,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296522,"Failed":false,"Killed":false,"Accumulables":[{"ID":66,"Name":"data size","Update":"40","Value":"40","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":75,"Name":"shuffle bytes written","Update":"84","Value":"84","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":76,"Name":"shuffle records written","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":77,"Name":"shuffle write time","Update":"3133081","Value":"3133081","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":78,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":79,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1418397,"Value":1418397,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.executorRunTime","Update":6,"Value":6,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.executorCpuTime","Update":6708366,"Value":6708366,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.resultSize","Update":1828,"Value":1828,"Internal":true,"Count Failed Values":true},{"ID":97,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":84,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":98,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":99,"Name":"internal.metrics.shuffle.write.writeTime","Update":3133081,"Value":3133081,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1418397,"Executor Run Time":6,"Executor CPU Time":6708366,"Peak Execution Memory":0,"Result Size":1828,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":84,"Shuffle Write Time":3133081,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"processLine at CliDriver.java:336","Number of Tasks":1,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"Exchange\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"LocalTableScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"11\",\"name\":\"LocalTableScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1714515296510,"Completion Time":1714515296522,"Accumulables":[{"ID":66,"Name":"data size","Value":"40","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":75,"Name":"shuffle bytes written","Value":"84","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":76,"Name":"shuffle records written","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":77,"Name":"shuffle write time","Value":"3133081","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":78,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":79,"Name":"internal.metrics.executorDeserializeTime","Value":1,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.executorDeserializeCpuTime","Value":1418397,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.executorRunTime","Value":6,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.executorCpuTime","Value":6708366,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.resultSize","Value":1828,"Internal":true,"Count Failed Values":true},{"ID":97,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":84,"Internal":true,"Count Failed Values":true},{"ID":98,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":1,"Internal":true,"Count Failed Values":true},{"ID":99,"Name":"internal.metrics.shuffle.write.writeTime","Value":3133081,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":2,"Completion Time":1714515296523,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate","executionId":6,"physicalPlanDescription":"== Physical Plan ==\nAppendData (5)\n+- AdaptiveSparkPlan (4)\n +- == Final Plan ==\n ShuffleQueryStage (3), Statistics(sizeInBytes=40.0 B, rowCount=1)\n +- Exchange (2)\n +- LocalTableScan (1)\n +- == Initial Plan ==\n Exchange (2)\n +- LocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [2]: [creator#13, projectname#14]\nArguments: [creator#13, projectname#14]\n\n(2) Exchange\nInput [2]: [creator#13, projectname#14]\nArguments: hashpartitioning(creator#13, 200), REPARTITION_BY_NUM, [plan_id=42]\n\n(3) ShuffleQueryStage\nOutput [2]: [creator#13, projectname#14]\nArguments: 0\n\n(4) AdaptiveSparkPlan\nOutput [2]: [creator#13, projectname#14]\nArguments: isFinalPlan=true\n\n(5) AppendData\nInput [2]: [creator#13, projectname#14]\nArguments: org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$2253/0x0000775334f5b4c0@72e89ce2, IcebergWrite(table=local.projects, format=PARQUET)\n\n","sparkPlanInfo":{"nodeName":"AppendData","simpleString":"AppendData org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$2253/0x0000775334f5b4c0@72e89ce2, IcebergWrite(table=local.projects, format=PARQUET)","children":[{"nodeName":"AdaptiveSparkPlan","simpleString":"AdaptiveSparkPlan isFinalPlan=true","children":[{"nodeName":"ShuffleQueryStage","simpleString":"ShuffleQueryStage 0","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(creator#13, 200), REPARTITION_BY_NUM, [plan_id=42]","children":[{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [creator#13, projectname#14]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":78,"metricType":"sum"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":76,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":77,"metricType":"nsTiming"},{"name":"records read","accumulatorId":74,"metricType":"sum"},{"name":"local bytes read","accumulatorId":72,"metricType":"size"},{"name":"fetch wait time","accumulatorId":73,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":70,"metricType":"size"},{"name":"local blocks read","accumulatorId":69,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":68,"metricType":"sum"},{"name":"data size","accumulatorId":66,"metricType":"size"},{"name":"number of partitions","accumulatorId":67,"metricType":"sum"},{"name":"remote bytes read to disk","accumulatorId":71,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":75,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}} +{"Event":"SparkListenerJobStart","Job ID":3,"Submission Time":1714515296531,"Stage Infos":[{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"processLine at CliDriver.java:336","Number of Tasks":200,"RDD Info":[{"RDD ID":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"13\",\"name\":\"Exchange\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"UNORDERED","Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[4],"Details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"processLine at CliDriver.java:336","Number of Tasks":1,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"Exchange\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"LocalTableScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"11\",\"name\":\"LocalTableScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[5,4],"Properties":{"spark.sql.warehouse.dir":"file:/home/holden/repos/sparklens/spark-warehouse","spark.executor.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.driver.host":"10.69.200.180","spark.eventLog.enabled":"true","spark.driver.port":"37317","__fetch_continuous_blocks_in_batch_enabled":"true","spark.sql.catalog.local.type":"hadoop","spark.jars":"","spark.sql.catalog.spark_catalog":"org.apache.iceberg.spark.SparkSessionCatalog","spark.sql.datetime.java8API.enabled":"true","spark.sql.hive.version":"2.3.9","spark.app.name":"fresh","spark.submit.pyFiles":"","spark.app.submitTime":"1714515291527","spark.job.description":"\nINSERT INTO local.projects VALUES(\"krisnova\", \"aurae\")","spark.app.startTime":"1714515291883","spark.executor.id":"driver","spark.sql.catalog.local.warehouse":"/home/holden/repos/sparklens/warehouse","spark.driver.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.sql.catalog.local":"org.apache.iceberg.spark.SparkCatalog","spark.submit.deployMode":"client","spark.master":"local[5]","spark.sql.execution.id":"6","spark.sql.catalogImplementation":"hive","spark.sql.catalog.spark_catalog.type":"hive","spark.app.id":"local-1714515292260"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"processLine at CliDriver.java:336","Number of Tasks":200,"RDD Info":[{"RDD ID":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"13\",\"name\":\"Exchange\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"UNORDERED","Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[4],"Details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1714515296535,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/holden/repos/sparklens/spark-warehouse","spark.executor.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.driver.host":"10.69.200.180","spark.eventLog.enabled":"true","spark.driver.port":"37317","__fetch_continuous_blocks_in_batch_enabled":"true","spark.sql.catalog.local.type":"hadoop","spark.jars":"","spark.sql.catalog.spark_catalog":"org.apache.iceberg.spark.SparkSessionCatalog","spark.sql.datetime.java8API.enabled":"true","spark.sql.hive.version":"2.3.9","spark.app.name":"fresh","spark.submit.pyFiles":"","spark.app.submitTime":"1714515291527","spark.job.description":"\nINSERT INTO local.projects VALUES(\"krisnova\", \"aurae\")","spark.app.startTime":"1714515291883","spark.executor.id":"driver","spark.sql.catalog.local.warehouse":"/home/holden/repos/sparklens/warehouse","spark.driver.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.sql.catalog.local":"org.apache.iceberg.spark.SparkCatalog","spark.submit.deployMode":"client","spark.master":"local[5]","spark.sql.execution.id":"6","spark.sql.catalogImplementation":"hive","spark.sql.catalog.spark_catalog.type":"hive","spark.app.id":"local-1714515292260"}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":202,"Index":68,"Attempt":0,"Partition ID":68,"Launch Time":1714515296540,"Executor ID":"driver","Host":"10.69.200.180","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":203,"Index":0,"Attempt":0,"Partition ID":0,"Launch Time":1714515296540,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":204,"Index":1,"Attempt":0,"Partition ID":1,"Launch Time":1714515296540,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":205,"Index":2,"Attempt":0,"Partition ID":2,"Launch Time":1714515296541,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":206,"Index":3,"Attempt":0,"Partition ID":3,"Launch Time":1714515296541,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":207,"Index":4,"Attempt":0,"Partition ID":4,"Launch Time":1714515296545,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":204,"Index":1,"Attempt":0,"Partition ID":1,"Launch Time":1714515296540,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296546,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1690222,"Value":1690222,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":2170768,"Value":2170768,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":2513,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1690222,"Executor Run Time":2,"Executor CPU Time":2170768,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":208,"Index":5,"Attempt":0,"Partition ID":5,"Launch Time":1714515296546,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":205,"Index":2,"Attempt":0,"Partition ID":2,"Launch Time":1714515296541,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296546,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1952140,"Value":3642362,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1386498,"Value":3557266,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":5026,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1952140,"Executor Run Time":1,"Executor CPU Time":1386498,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":209,"Index":6,"Attempt":0,"Partition ID":6,"Launch Time":1714515296546,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":203,"Index":0,"Attempt":0,"Partition ID":0,"Launch Time":1714515296540,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296551,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1667633,"Value":5309995,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":2,"Value":5,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1616886,"Value":5174152,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":7539,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1667633,"Executor Run Time":2,"Executor CPU Time":1616886,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":210,"Index":7,"Attempt":0,"Partition ID":7,"Launch Time":1714515296551,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":206,"Index":3,"Attempt":0,"Partition ID":3,"Launch Time":1714515296541,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296552,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":4,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1792508,"Value":7102503,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":2,"Value":7,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1473446,"Value":6647598,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":10052,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1792508,"Executor Run Time":2,"Executor CPU Time":1473446,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":211,"Index":8,"Attempt":0,"Partition ID":8,"Launch Time":1714515296555,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":208,"Index":5,"Attempt":0,"Partition ID":5,"Launch Time":1714515296546,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296555,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":10,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1809897,"Value":8912400,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":8,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1784269,"Value":8431867,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2556,"Value":12608,"Internal":true,"Count Failed Values":true},{"ID":109,"Name":"internal.metrics.jvmGCTime","Update":4,"Value":4,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":1809897,"Executor Run Time":1,"Executor CPU Time":1784269,"Peak Execution Memory":0,"Result Size":2556,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":212,"Index":9,"Attempt":0,"Partition ID":9,"Launch Time":1714515296555,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":213,"Index":10,"Attempt":0,"Partition ID":10,"Launch Time":1714515296555,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":207,"Index":4,"Attempt":0,"Partition ID":4,"Launch Time":1714515296545,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296555,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":11,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1979028,"Value":10891428,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":6,"Value":14,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1919730,"Value":10351597,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2556,"Value":15164,"Internal":true,"Count Failed Values":true},{"ID":109,"Name":"internal.metrics.jvmGCTime","Update":4,"Value":8,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1979028,"Executor Run Time":6,"Executor CPU Time":1919730,"Peak Execution Memory":0,"Result Size":2556,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":214,"Index":11,"Attempt":0,"Partition ID":11,"Launch Time":1714515296556,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":210,"Index":7,"Attempt":0,"Partition ID":7,"Launch Time":1714515296551,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296556,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":12,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1297793,"Value":12189221,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":15,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1406318,"Value":11757915,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":17677,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1297793,"Executor Run Time":1,"Executor CPU Time":1406318,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":209,"Index":6,"Attempt":0,"Partition ID":6,"Launch Time":1714515296546,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296556,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":13,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1694702,"Value":13883923,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":16,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1572630,"Value":13330545,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":20190,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1694702,"Executor Run Time":1,"Executor CPU Time":1572630,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":215,"Index":12,"Attempt":0,"Partition ID":12,"Launch Time":1714515296558,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":216,"Index":13,"Attempt":0,"Partition ID":13,"Launch Time":1714515296558,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":212,"Index":9,"Attempt":0,"Partition ID":9,"Launch Time":1714515296555,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296558,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":14,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1170090,"Value":15054013,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":946421,"Value":14276966,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":22660,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1170090,"Executor Run Time":0,"Executor CPU Time":946421,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":211,"Index":8,"Attempt":0,"Partition ID":8,"Launch Time":1714515296555,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296558,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":15,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1367910,"Value":16421923,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":17,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1064955,"Value":15341921,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":25173,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1367910,"Executor Run Time":1,"Executor CPU Time":1064955,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":217,"Index":14,"Attempt":0,"Partition ID":14,"Launch Time":1714515296559,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":218,"Index":15,"Attempt":0,"Partition ID":15,"Launch Time":1714515296559,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":214,"Index":11,"Attempt":0,"Partition ID":11,"Launch Time":1714515296556,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296559,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":16,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1245236,"Value":17667159,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":18,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1194539,"Value":16536460,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":27686,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1245236,"Executor Run Time":1,"Executor CPU Time":1194539,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":213,"Index":10,"Attempt":0,"Partition ID":10,"Launch Time":1714515296555,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296559,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":17,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1506713,"Value":19173872,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":19,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1360240,"Value":17896700,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":30199,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1506713,"Executor Run Time":1,"Executor CPU Time":1360240,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":219,"Index":16,"Attempt":0,"Partition ID":16,"Launch Time":1714515296561,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":220,"Index":17,"Attempt":0,"Partition ID":17,"Launch Time":1714515296561,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":216,"Index":13,"Attempt":0,"Partition ID":13,"Launch Time":1714515296558,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296561,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":18,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1103863,"Value":20277735,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":867096,"Value":18763796,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":32669,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1103863,"Executor Run Time":0,"Executor CPU Time":867096,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":215,"Index":12,"Attempt":0,"Partition ID":12,"Launch Time":1714515296558,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296561,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":19,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1251615,"Value":21529350,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":975920,"Value":19739716,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":35139,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1251615,"Executor Run Time":0,"Executor CPU Time":975920,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":221,"Index":18,"Attempt":0,"Partition ID":18,"Launch Time":1714515296562,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":222,"Index":19,"Attempt":0,"Partition ID":19,"Launch Time":1714515296562,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":218,"Index":15,"Attempt":0,"Partition ID":15,"Launch Time":1714515296559,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296562,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":20,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1129333,"Value":22658683,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":20,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1055525,"Value":20795241,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":37652,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1129333,"Executor Run Time":1,"Executor CPU Time":1055525,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":217,"Index":14,"Attempt":0,"Partition ID":14,"Launch Time":1714515296559,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296562,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":21,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1331351,"Value":23990034,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":21,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1205178,"Value":22000419,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":40165,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1331351,"Executor Run Time":1,"Executor CPU Time":1205178,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":223,"Index":20,"Attempt":0,"Partition ID":20,"Launch Time":1714515296564,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":220,"Index":17,"Attempt":0,"Partition ID":17,"Launch Time":1714515296561,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296564,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":979869,"Value":24969903,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":814548,"Value":22814967,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":42592,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":979869,"Executor Run Time":0,"Executor CPU Time":814548,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":224,"Index":21,"Attempt":0,"Partition ID":21,"Launch Time":1714515296564,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":219,"Index":16,"Attempt":0,"Partition ID":16,"Launch Time":1714515296561,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296564,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":22,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1264436,"Value":26234339,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":841156,"Value":23656123,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":45062,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1264436,"Executor Run Time":0,"Executor CPU Time":841156,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":225,"Index":22,"Attempt":0,"Partition ID":22,"Launch Time":1714515296565,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":222,"Index":19,"Attempt":0,"Partition ID":19,"Launch Time":1714515296562,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296565,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":23,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1086914,"Value":27321253,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":969860,"Value":24625983,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":47532,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1086914,"Executor Run Time":0,"Executor CPU Time":969860,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":226,"Index":23,"Attempt":0,"Partition ID":23,"Launch Time":1714515296565,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":221,"Index":18,"Attempt":0,"Partition ID":18,"Launch Time":1714515296562,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296565,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":24,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1321302,"Value":28642555,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":22,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1091444,"Value":25717427,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":50045,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1321302,"Executor Run Time":1,"Executor CPU Time":1091444,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":227,"Index":24,"Attempt":0,"Partition ID":24,"Launch Time":1714515296566,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":228,"Index":25,"Attempt":0,"Partition ID":25,"Launch Time":1714515296566,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":229,"Index":26,"Attempt":0,"Partition ID":26,"Launch Time":1714515296566,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":202,"Index":68,"Attempt":0,"Partition ID":68,"Launch Time":1714515296540,"Executor ID":"driver","Host":"10.69.200.180","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296566,"Failed":false,"Killed":false,"Accumulables":[{"ID":69,"Name":"local blocks read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":72,"Name":"local bytes read","Update":"84","Value":"84","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":73,"Name":"fetch wait time","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"records read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":25,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1780298,"Value":30422853,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":22,"Value":44,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":10310218,"Value":36027645,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":5181,"Value":55226,"Internal":true,"Count Failed Values":true},{"ID":109,"Name":"internal.metrics.jvmGCTime","Update":4,"Value":12,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":84,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":127,"Name":"internal.metrics.output.bytesWritten","Update":703,"Value":703,"Internal":true,"Count Failed Values":true},{"ID":128,"Name":"internal.metrics.output.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1780298,"Executor Run Time":22,"Executor CPU Time":10310218,"Peak Execution Memory":0,"Result Size":5181,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":84,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":703,"Records Written":1},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":224,"Index":21,"Attempt":0,"Partition ID":21,"Launch Time":1714515296564,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296566,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":768381,"Value":31191234,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":666895,"Value":36694540,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":57653,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":768381,"Executor Run Time":0,"Executor CPU Time":666895,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":223,"Index":20,"Attempt":0,"Partition ID":20,"Launch Time":1714515296564,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296567,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":26,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1116123,"Value":32307357,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":953281,"Value":37647821,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":60123,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1116123,"Executor Run Time":0,"Executor CPU Time":953281,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":230,"Index":27,"Attempt":0,"Partition ID":27,"Launch Time":1714515296568,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":227,"Index":24,"Attempt":0,"Partition ID":24,"Launch Time":1714515296566,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296568,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":982299,"Value":33289656,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":831417,"Value":38479238,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":62550,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":982299,"Executor Run Time":0,"Executor CPU Time":831417,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":231,"Index":28,"Attempt":0,"Partition ID":28,"Launch Time":1714515296569,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":226,"Index":23,"Attempt":0,"Partition ID":23,"Launch Time":1714515296565,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296569,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":27,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1393118,"Value":34682774,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":45,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1108723,"Value":39587961,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":65063,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1393118,"Executor Run Time":1,"Executor CPU Time":1108723,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":232,"Index":29,"Attempt":0,"Partition ID":29,"Launch Time":1714515296569,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":225,"Index":22,"Attempt":0,"Partition ID":22,"Launch Time":1714515296565,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296569,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":28,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1665884,"Value":36348658,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":46,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1178700,"Value":40766661,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":67576,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1665884,"Executor Run Time":1,"Executor CPU Time":1178700,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":233,"Index":30,"Attempt":0,"Partition ID":30,"Launch Time":1714515296570,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":229,"Index":26,"Attempt":0,"Partition ID":26,"Launch Time":1714515296566,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296570,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":29,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1084575,"Value":37433233,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":963999,"Value":41730660,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":70046,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1084575,"Executor Run Time":0,"Executor CPU Time":963999,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":234,"Index":31,"Attempt":0,"Partition ID":31,"Launch Time":1714515296570,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":228,"Index":25,"Attempt":0,"Partition ID":25,"Launch Time":1714515296566,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296570,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":30,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1188369,"Value":38621602,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":47,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1113243,"Value":42843903,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":72559,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1188369,"Executor Run Time":1,"Executor CPU Time":1113243,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":235,"Index":32,"Attempt":0,"Partition ID":32,"Launch Time":1714515296571,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":230,"Index":27,"Attempt":0,"Partition ID":27,"Launch Time":1714515296568,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296571,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":31,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1277984,"Value":39899586,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":48,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1048956,"Value":43892859,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":75072,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1277984,"Executor Run Time":1,"Executor CPU Time":1048956,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":236,"Index":33,"Attempt":0,"Partition ID":33,"Launch Time":1714515296572,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":231,"Index":28,"Attempt":0,"Partition ID":28,"Launch Time":1714515296569,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296572,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":32,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1190339,"Value":41089925,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":934872,"Value":44827731,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":77542,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1190339,"Executor Run Time":0,"Executor CPU Time":934872,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":237,"Index":34,"Attempt":0,"Partition ID":34,"Launch Time":1714515296572,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":232,"Index":29,"Attempt":0,"Partition ID":29,"Launch Time":1714515296569,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296572,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":33,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1205228,"Value":42295153,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":850386,"Value":45678117,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":80012,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1205228,"Executor Run Time":0,"Executor CPU Time":850386,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":238,"Index":35,"Attempt":0,"Partition ID":35,"Launch Time":1714515296573,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":233,"Index":30,"Attempt":0,"Partition ID":30,"Launch Time":1714515296570,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296573,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":34,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1385559,"Value":43680712,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":49,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1335761,"Value":47013878,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":82525,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1385559,"Executor Run Time":1,"Executor CPU Time":1335761,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":239,"Index":36,"Attempt":0,"Partition ID":36,"Launch Time":1714515296574,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":234,"Index":31,"Attempt":0,"Partition ID":31,"Launch Time":1714515296570,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296574,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":35,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1319323,"Value":45000035,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":50,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1088354,"Value":48102232,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":85038,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1319323,"Executor Run Time":1,"Executor CPU Time":1088354,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":240,"Index":37,"Attempt":0,"Partition ID":37,"Launch Time":1714515296574,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":241,"Index":38,"Attempt":0,"Partition ID":38,"Launch Time":1714515296574,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":236,"Index":33,"Attempt":0,"Partition ID":33,"Launch Time":1714515296572,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296575,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":849617,"Value":45849652,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":850315,"Value":48952547,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":87465,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":849617,"Executor Run Time":0,"Executor CPU Time":850315,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":235,"Index":32,"Attempt":0,"Partition ID":32,"Launch Time":1714515296571,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296575,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":36,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1229405,"Value":47079057,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":51,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1025348,"Value":49977895,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":89978,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1229405,"Executor Run Time":1,"Executor CPU Time":1025348,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":242,"Index":39,"Attempt":0,"Partition ID":39,"Launch Time":1714515296575,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":237,"Index":34,"Attempt":0,"Partition ID":34,"Launch Time":1714515296572,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296575,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":37,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1066205,"Value":48145262,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":959840,"Value":50937735,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":92448,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1066205,"Executor Run Time":0,"Executor CPU Time":959840,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":243,"Index":40,"Attempt":0,"Partition ID":40,"Launch Time":1714515296576,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":238,"Index":35,"Attempt":0,"Partition ID":35,"Launch Time":1714515296573,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296576,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":38,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1202188,"Value":49347450,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":52,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1184879,"Value":52122614,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":94961,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1202188,"Executor Run Time":1,"Executor CPU Time":1184879,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":244,"Index":41,"Attempt":0,"Partition ID":41,"Launch Time":1714515296577,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":239,"Index":36,"Attempt":0,"Partition ID":36,"Launch Time":1714515296574,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296577,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":39,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1349671,"Value":50697121,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":53,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":991318,"Value":53113932,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":97474,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1349671,"Executor Run Time":1,"Executor CPU Time":991318,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":245,"Index":42,"Attempt":0,"Partition ID":42,"Launch Time":1714515296577,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":240,"Index":37,"Attempt":0,"Partition ID":37,"Launch Time":1714515296574,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296577,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":40,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1125772,"Value":51822893,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":54,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1016228,"Value":54130160,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":99987,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1125772,"Executor Run Time":1,"Executor CPU Time":1016228,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":246,"Index":43,"Attempt":0,"Partition ID":43,"Launch Time":1714515296577,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":242,"Index":39,"Attempt":0,"Partition ID":39,"Launch Time":1714515296575,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296578,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":813158,"Value":52636051,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":790599,"Value":54920759,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":102414,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":813158,"Executor Run Time":0,"Executor CPU Time":790599,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":247,"Index":44,"Attempt":0,"Partition ID":44,"Launch Time":1714515296578,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":241,"Index":38,"Attempt":0,"Partition ID":38,"Launch Time":1714515296574,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296578,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":41,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1074515,"Value":53710566,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":901113,"Value":55821872,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":104884,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1074515,"Executor Run Time":0,"Executor CPU Time":901113,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":248,"Index":45,"Attempt":0,"Partition ID":45,"Launch Time":1714515296579,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":243,"Index":40,"Attempt":0,"Partition ID":40,"Launch Time":1714515296576,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296580,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":42,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1321372,"Value":55031938,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":55,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1076794,"Value":56898666,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":107397,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1321372,"Executor Run Time":1,"Executor CPU Time":1076794,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":249,"Index":46,"Attempt":0,"Partition ID":46,"Launch Time":1714515296580,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":246,"Index":43,"Attempt":0,"Partition ID":43,"Launch Time":1714515296577,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296580,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":825657,"Value":55857595,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":825318,"Value":57723984,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":109824,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":825657,"Executor Run Time":0,"Executor CPU Time":825318,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":250,"Index":47,"Attempt":0,"Partition ID":47,"Launch Time":1714515296580,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":251,"Index":48,"Attempt":0,"Partition ID":48,"Launch Time":1714515296581,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":245,"Index":42,"Attempt":0,"Partition ID":42,"Launch Time":1714515296577,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296581,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":43,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1205778,"Value":57063373,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":976679,"Value":58700663,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":112294,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1205778,"Executor Run Time":0,"Executor CPU Time":976679,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":244,"Index":41,"Attempt":0,"Partition ID":41,"Launch Time":1714515296577,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296581,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":44,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1409557,"Value":58472930,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":56,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1122473,"Value":59823136,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":114807,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1409557,"Executor Run Time":1,"Executor CPU Time":1122473,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":252,"Index":49,"Attempt":0,"Partition ID":49,"Launch Time":1714515296581,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":247,"Index":44,"Attempt":0,"Partition ID":44,"Launch Time":1714515296578,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296581,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":983179,"Value":59456109,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":908513,"Value":60731649,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":117234,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":983179,"Executor Run Time":0,"Executor CPU Time":908513,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":253,"Index":50,"Attempt":0,"Partition ID":50,"Launch Time":1714515296582,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":248,"Index":45,"Attempt":0,"Partition ID":45,"Launch Time":1714515296579,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296582,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":45,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":993179,"Value":60449288,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":716493,"Value":61448142,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":119704,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":993179,"Executor Run Time":0,"Executor CPU Time":716493,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":254,"Index":51,"Attempt":0,"Partition ID":51,"Launch Time":1714515296583,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":251,"Index":48,"Attempt":0,"Partition ID":48,"Launch Time":1714515296581,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296584,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":46,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1192859,"Value":61642147,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":974570,"Value":62422712,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":122174,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1192859,"Executor Run Time":0,"Executor CPU Time":974570,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":255,"Index":52,"Attempt":0,"Partition ID":52,"Launch Time":1714515296584,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":250,"Index":47,"Attempt":0,"Partition ID":47,"Launch Time":1714515296580,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296584,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":47,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1301812,"Value":62943959,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":57,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1083845,"Value":63506557,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":124687,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1301812,"Executor Run Time":1,"Executor CPU Time":1083845,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":256,"Index":53,"Attempt":0,"Partition ID":53,"Launch Time":1714515296584,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":249,"Index":46,"Attempt":0,"Partition ID":46,"Launch Time":1714515296580,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296584,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":48,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1530031,"Value":64473990,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":58,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1344310,"Value":64850867,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":127200,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1530031,"Executor Run Time":1,"Executor CPU Time":1344310,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":257,"Index":54,"Attempt":0,"Partition ID":54,"Launch Time":1714515296584,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":258,"Index":55,"Attempt":0,"Partition ID":55,"Launch Time":1714515296585,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":253,"Index":50,"Attempt":0,"Partition ID":50,"Launch Time":1714515296582,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296585,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":719633,"Value":65193623,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":811169,"Value":65662036,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":129627,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":719633,"Executor Run Time":0,"Executor CPU Time":811169,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":252,"Index":49,"Attempt":0,"Partition ID":49,"Launch Time":1714515296581,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296585,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":49,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1289173,"Value":66482796,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":59,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1107594,"Value":66769630,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":132140,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1289173,"Executor Run Time":1,"Executor CPU Time":1107594,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":259,"Index":56,"Attempt":0,"Partition ID":56,"Launch Time":1714515296588,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":256,"Index":53,"Attempt":0,"Partition ID":53,"Launch Time":1714515296584,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296588,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":50,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1573089,"Value":68055885,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1444595,"Value":68214225,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":134653,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1573089,"Executor Run Time":1,"Executor CPU Time":1444595,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":260,"Index":57,"Attempt":0,"Partition ID":57,"Launch Time":1714515296588,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":255,"Index":52,"Attempt":0,"Partition ID":52,"Launch Time":1714515296584,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296589,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":52,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1799606,"Value":69855491,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":61,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1442055,"Value":69656280,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":137166,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1799606,"Executor Run Time":1,"Executor CPU Time":1442055,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":261,"Index":58,"Attempt":0,"Partition ID":58,"Launch Time":1714515296589,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":262,"Index":59,"Attempt":0,"Partition ID":59,"Launch Time":1714515296589,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":254,"Index":51,"Attempt":0,"Partition ID":51,"Launch Time":1714515296583,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296589,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":53,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1830616,"Value":71686107,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":2,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1746780,"Value":71403060,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":139679,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1830616,"Executor Run Time":2,"Executor CPU Time":1746780,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":263,"Index":60,"Attempt":0,"Partition ID":60,"Launch Time":1714515296589,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":257,"Index":54,"Attempt":0,"Partition ID":54,"Launch Time":1714515296584,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296590,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":55,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1851664,"Value":73537771,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":64,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1249495,"Value":72652555,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":142192,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1851664,"Executor Run Time":1,"Executor CPU Time":1249495,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":258,"Index":55,"Attempt":0,"Partition ID":55,"Launch Time":1714515296585,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296590,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":56,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1687293,"Value":75225064,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":65,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1336631,"Value":73989186,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":144705,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1687293,"Executor Run Time":1,"Executor CPU Time":1336631,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":264,"Index":61,"Attempt":0,"Partition ID":61,"Launch Time":1714515296591,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":260,"Index":57,"Attempt":0,"Partition ID":57,"Launch Time":1714515296588,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296592,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":57,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1324144,"Value":76549208,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":869014,"Value":74858200,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":147175,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1324144,"Executor Run Time":0,"Executor CPU Time":869014,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":265,"Index":62,"Attempt":0,"Partition ID":62,"Launch Time":1714515296592,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":266,"Index":63,"Attempt":0,"Partition ID":63,"Launch Time":1714515296592,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":259,"Index":56,"Attempt":0,"Partition ID":56,"Launch Time":1714515296588,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296592,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":58,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1435166,"Value":77984374,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":66,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1146102,"Value":76004302,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":149688,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1435166,"Executor Run Time":1,"Executor CPU Time":1146102,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":267,"Index":64,"Attempt":0,"Partition ID":64,"Launch Time":1714515296592,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":268,"Index":65,"Attempt":0,"Partition ID":65,"Launch Time":1714515296592,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":263,"Index":60,"Attempt":0,"Partition ID":60,"Launch Time":1714515296589,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296592,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":958871,"Value":78943245,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":926762,"Value":76931064,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":152115,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":958871,"Executor Run Time":0,"Executor CPU Time":926762,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":261,"Index":58,"Attempt":0,"Partition ID":58,"Launch Time":1714515296589,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296593,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":59,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1386768,"Value":80330013,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":988249,"Value":77919313,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":154585,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1386768,"Executor Run Time":0,"Executor CPU Time":988249,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":262,"Index":59,"Attempt":0,"Partition ID":59,"Launch Time":1714515296589,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296593,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1186928,"Value":81516941,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":67,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":950031,"Value":78869344,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":157098,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1186928,"Executor Run Time":1,"Executor CPU Time":950031,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":269,"Index":66,"Attempt":0,"Partition ID":66,"Launch Time":1714515296594,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":270,"Index":67,"Attempt":0,"Partition ID":67,"Launch Time":1714515296594,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":264,"Index":61,"Attempt":0,"Partition ID":61,"Launch Time":1714515296591,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296595,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":61,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1217877,"Value":82734818,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":68,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1046798,"Value":79916142,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":159611,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1217877,"Executor Run Time":1,"Executor CPU Time":1046798,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":267,"Index":64,"Attempt":0,"Partition ID":64,"Launch Time":1714515296592,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296595,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":62,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1048327,"Value":83783145,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":780439,"Value":80696581,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":162081,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1048327,"Executor Run Time":0,"Executor CPU Time":780439,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":271,"Index":69,"Attempt":0,"Partition ID":69,"Launch Time":1714515296595,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":272,"Index":70,"Attempt":0,"Partition ID":70,"Launch Time":1714515296595,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":265,"Index":62,"Attempt":0,"Partition ID":62,"Launch Time":1714515296592,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296595,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1339703,"Value":85122848,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":69,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1047536,"Value":81744117,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":164594,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1339703,"Executor Run Time":1,"Executor CPU Time":1047536,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":266,"Index":63,"Attempt":0,"Partition ID":63,"Launch Time":1714515296592,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296595,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":64,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1224167,"Value":86347015,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":70,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":986919,"Value":82731036,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":167107,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1224167,"Executor Run Time":1,"Executor CPU Time":986919,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":273,"Index":71,"Attempt":0,"Partition ID":71,"Launch Time":1714515296596,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":268,"Index":65,"Attempt":0,"Partition ID":65,"Launch Time":1714515296592,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296596,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":65,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1220517,"Value":87567532,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":71,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1159961,"Value":83890997,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":169620,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1220517,"Executor Run Time":1,"Executor CPU Time":1159961,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":274,"Index":72,"Attempt":0,"Partition ID":72,"Launch Time":1714515296597,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":275,"Index":73,"Attempt":0,"Partition ID":73,"Launch Time":1714515296598,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":269,"Index":66,"Attempt":0,"Partition ID":66,"Launch Time":1714515296594,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296598,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":66,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1441607,"Value":89009139,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":72,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1113292,"Value":85004289,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":172133,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1441607,"Executor Run Time":1,"Executor CPU Time":1113292,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":270,"Index":67,"Attempt":0,"Partition ID":67,"Launch Time":1714515296594,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296598,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":67,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1230776,"Value":90239915,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":73,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1097063,"Value":86101352,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":174646,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1230776,"Executor Run Time":1,"Executor CPU Time":1097063,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":276,"Index":74,"Attempt":0,"Partition ID":74,"Launch Time":1714515296598,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":271,"Index":69,"Attempt":0,"Partition ID":69,"Launch Time":1714515296595,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296598,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":68,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1286194,"Value":91526109,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":968429,"Value":87069781,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":177116,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1286194,"Executor Run Time":0,"Executor CPU Time":968429,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":277,"Index":75,"Attempt":0,"Partition ID":75,"Launch Time":1714515296598,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":278,"Index":76,"Attempt":0,"Partition ID":76,"Launch Time":1714515296598,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":272,"Index":70,"Attempt":0,"Partition ID":70,"Launch Time":1714515296595,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296599,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":69,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1166689,"Value":92692798,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":911744,"Value":87981525,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":179586,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1166689,"Executor Run Time":0,"Executor CPU Time":911744,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":273,"Index":71,"Attempt":0,"Partition ID":71,"Launch Time":1714515296596,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296599,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":70,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1236476,"Value":93929274,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":999468,"Value":88980993,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":182056,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1236476,"Executor Run Time":0,"Executor CPU Time":999468,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":279,"Index":77,"Attempt":0,"Partition ID":77,"Launch Time":1714515296600,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":274,"Index":72,"Attempt":0,"Partition ID":72,"Launch Time":1714515296597,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296600,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":71,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1078254,"Value":95007528,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":743852,"Value":89724845,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":184526,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1078254,"Executor Run Time":0,"Executor CPU Time":743852,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":280,"Index":78,"Attempt":0,"Partition ID":78,"Launch Time":1714515296600,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":281,"Index":79,"Attempt":0,"Partition ID":79,"Launch Time":1714515296600,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":275,"Index":73,"Attempt":0,"Partition ID":73,"Launch Time":1714515296598,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296601,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":72,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1184369,"Value":96191897,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":74,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1016238,"Value":90741083,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":187039,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1184369,"Executor Run Time":1,"Executor CPU Time":1016238,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":276,"Index":74,"Attempt":0,"Partition ID":74,"Launch Time":1714515296598,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296601,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":73,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1074205,"Value":97266102,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":898463,"Value":91639546,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":189509,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1074205,"Executor Run Time":0,"Executor CPU Time":898463,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":282,"Index":80,"Attempt":0,"Partition ID":80,"Launch Time":1714515296601,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":283,"Index":81,"Attempt":0,"Partition ID":81,"Launch Time":1714515296601,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":278,"Index":76,"Attempt":0,"Partition ID":76,"Launch Time":1714515296598,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296601,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":74,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1119763,"Value":98385865,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":728252,"Value":92367798,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":191979,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1119763,"Executor Run Time":0,"Executor CPU Time":728252,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":277,"Index":75,"Attempt":0,"Partition ID":75,"Launch Time":1714515296598,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296601,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":75,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1304783,"Value":99690648,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":960770,"Value":93328568,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":194449,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1304783,"Executor Run Time":0,"Executor CPU Time":960770,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":284,"Index":82,"Attempt":0,"Partition ID":82,"Launch Time":1714515296601,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":279,"Index":77,"Attempt":0,"Partition ID":77,"Launch Time":1714515296600,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296602,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":784029,"Value":100474677,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":644377,"Value":93972945,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":196876,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":784029,"Executor Run Time":0,"Executor CPU Time":644377,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":285,"Index":83,"Attempt":0,"Partition ID":83,"Launch Time":1714515296603,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":280,"Index":78,"Attempt":0,"Partition ID":78,"Launch Time":1714515296600,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296603,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":76,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1084805,"Value":101559482,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":852536,"Value":94825481,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":199346,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1084805,"Executor Run Time":0,"Executor CPU Time":852536,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":286,"Index":84,"Attempt":0,"Partition ID":84,"Launch Time":1714515296603,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":287,"Index":85,"Attempt":0,"Partition ID":85,"Launch Time":1714515296604,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":281,"Index":79,"Attempt":0,"Partition ID":79,"Launch Time":1714515296600,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296604,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":77,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1209388,"Value":102768870,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":75,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1035566,"Value":95861047,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":201859,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1209388,"Executor Run Time":1,"Executor CPU Time":1035566,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":283,"Index":81,"Attempt":0,"Partition ID":81,"Launch Time":1714515296601,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296604,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":78,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1199278,"Value":103968148,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":833857,"Value":96694904,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":204329,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1199278,"Executor Run Time":0,"Executor CPU Time":833857,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":288,"Index":86,"Attempt":0,"Partition ID":86,"Launch Time":1714515296604,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":289,"Index":87,"Attempt":0,"Partition ID":87,"Launch Time":1714515296604,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":284,"Index":82,"Attempt":0,"Partition ID":82,"Launch Time":1714515296601,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296604,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":842587,"Value":104810735,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":950841,"Value":97645745,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":206756,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":842587,"Executor Run Time":0,"Executor CPU Time":950841,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":282,"Index":80,"Attempt":0,"Partition ID":80,"Launch Time":1714515296601,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296604,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":79,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1309143,"Value":106119878,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":76,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1090944,"Value":98736689,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":209269,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1309143,"Executor Run Time":1,"Executor CPU Time":1090944,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":290,"Index":88,"Attempt":0,"Partition ID":88,"Launch Time":1714515296605,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":291,"Index":89,"Attempt":0,"Partition ID":89,"Launch Time":1714515296606,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":285,"Index":83,"Attempt":0,"Partition ID":83,"Launch Time":1714515296603,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296606,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":80,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1100164,"Value":107220042,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":903533,"Value":99640222,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":211739,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1100164,"Executor Run Time":0,"Executor CPU Time":903533,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":286,"Index":84,"Attempt":0,"Partition ID":84,"Launch Time":1714515296603,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296606,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":914403,"Value":108134445,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":637857,"Value":100278079,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":214166,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":914403,"Executor Run Time":0,"Executor CPU Time":637857,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":292,"Index":90,"Attempt":0,"Partition ID":90,"Launch Time":1714515296606,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":288,"Index":86,"Attempt":0,"Partition ID":86,"Launch Time":1714515296604,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296606,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":865995,"Value":109000440,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":762851,"Value":101040930,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":216593,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":865995,"Executor Run Time":0,"Executor CPU Time":762851,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":293,"Index":91,"Attempt":0,"Partition ID":91,"Launch Time":1714515296607,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":287,"Index":85,"Attempt":0,"Partition ID":85,"Launch Time":1714515296604,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296607,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":81,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1108513,"Value":110108953,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":855006,"Value":101895936,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":219063,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1108513,"Executor Run Time":0,"Executor CPU Time":855006,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":294,"Index":92,"Attempt":0,"Partition ID":92,"Launch Time":1714515296607,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":289,"Index":87,"Attempt":0,"Partition ID":87,"Launch Time":1714515296604,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296607,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":82,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1242736,"Value":111351689,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":77,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1154141,"Value":103050077,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":221576,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1242736,"Executor Run Time":1,"Executor CPU Time":1154141,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":295,"Index":93,"Attempt":0,"Partition ID":93,"Launch Time":1714515296608,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":290,"Index":88,"Attempt":0,"Partition ID":88,"Launch Time":1714515296605,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296608,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":846256,"Value":112197945,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":764091,"Value":103814168,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":224003,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":846256,"Executor Run Time":0,"Executor CPU Time":764091,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":296,"Index":94,"Attempt":0,"Partition ID":94,"Launch Time":1714515296608,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":291,"Index":89,"Attempt":0,"Partition ID":89,"Launch Time":1714515296606,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296608,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":83,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1041657,"Value":113239602,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":852706,"Value":104666874,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":226473,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1041657,"Executor Run Time":0,"Executor CPU Time":852706,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":297,"Index":95,"Attempt":0,"Partition ID":95,"Launch Time":1714515296609,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":298,"Index":96,"Attempt":0,"Partition ID":96,"Launch Time":1714515296609,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":299,"Index":97,"Attempt":0,"Partition ID":97,"Launch Time":1714515296609,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":293,"Index":91,"Attempt":0,"Partition ID":91,"Launch Time":1714515296607,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296609,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":837976,"Value":114077578,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":599810,"Value":105266684,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":228900,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":837976,"Executor Run Time":0,"Executor CPU Time":599810,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":292,"Index":90,"Attempt":0,"Partition ID":90,"Launch Time":1714515296606,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296609,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1109293,"Value":115186871,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":882635,"Value":106149319,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":231370,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1109293,"Executor Run Time":0,"Executor CPU Time":882635,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":294,"Index":92,"Attempt":0,"Partition ID":92,"Launch Time":1714515296607,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296609,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":761080,"Value":115947951,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":640277,"Value":106789596,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":233797,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":761080,"Executor Run Time":0,"Executor CPU Time":640277,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":300,"Index":98,"Attempt":0,"Partition ID":98,"Launch Time":1714515296610,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":295,"Index":93,"Attempt":0,"Partition ID":93,"Launch Time":1714515296608,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296610,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":85,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1162179,"Value":117110130,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":901254,"Value":107690850,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":236267,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1162179,"Executor Run Time":0,"Executor CPU Time":901254,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":301,"Index":99,"Attempt":0,"Partition ID":99,"Launch Time":1714515296610,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":296,"Index":94,"Attempt":0,"Partition ID":94,"Launch Time":1714515296608,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296611,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":86,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1121662,"Value":118231792,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":735673,"Value":108426523,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":238737,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1121662,"Executor Run Time":0,"Executor CPU Time":735673,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":302,"Index":100,"Attempt":0,"Partition ID":100,"Launch Time":1714515296611,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":298,"Index":96,"Attempt":0,"Partition ID":96,"Launch Time":1714515296609,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296611,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":956461,"Value":119188253,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":641157,"Value":109067680,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":241164,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":956461,"Executor Run Time":0,"Executor CPU Time":641157,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":303,"Index":101,"Attempt":0,"Partition ID":101,"Launch Time":1714515296611,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":299,"Index":97,"Attempt":0,"Partition ID":97,"Launch Time":1714515296609,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296611,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":898084,"Value":120086337,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":659865,"Value":109727545,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":243591,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":898084,"Executor Run Time":0,"Executor CPU Time":659865,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":304,"Index":102,"Attempt":0,"Partition ID":102,"Launch Time":1714515296612,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":297,"Index":95,"Attempt":0,"Partition ID":95,"Launch Time":1714515296609,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296612,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":87,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1213367,"Value":121299704,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":907923,"Value":110635468,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":246061,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1213367,"Executor Run Time":0,"Executor CPU Time":907923,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":305,"Index":103,"Attempt":0,"Partition ID":103,"Launch Time":1714515296612,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":300,"Index":98,"Attempt":0,"Partition ID":98,"Launch Time":1714515296610,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296613,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":88,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1033317,"Value":122333021,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":789989,"Value":111425457,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":248531,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1033317,"Executor Run Time":0,"Executor CPU Time":789989,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":306,"Index":104,"Attempt":0,"Partition ID":104,"Launch Time":1714515296613,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":302,"Index":100,"Attempt":0,"Partition ID":100,"Launch Time":1714515296611,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296613,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":950122,"Value":123283143,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":680434,"Value":112105891,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":250958,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":950122,"Executor Run Time":0,"Executor CPU Time":680434,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":307,"Index":105,"Attempt":0,"Partition ID":105,"Launch Time":1714515296613,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":301,"Index":99,"Attempt":0,"Partition ID":99,"Launch Time":1714515296610,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296613,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":89,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1337042,"Value":124620185,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":953580,"Value":113059471,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":253428,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1337042,"Executor Run Time":0,"Executor CPU Time":953580,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":308,"Index":106,"Attempt":0,"Partition ID":106,"Launch Time":1714515296614,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":304,"Index":102,"Attempt":0,"Partition ID":102,"Launch Time":1714515296612,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296614,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":915313,"Value":125535498,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":656316,"Value":113715787,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":255855,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":915313,"Executor Run Time":0,"Executor CPU Time":656316,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":309,"Index":107,"Attempt":0,"Partition ID":107,"Launch Time":1714515296614,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":303,"Index":101,"Attempt":0,"Partition ID":101,"Launch Time":1714515296611,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296614,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":90,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1294783,"Value":126830281,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":952332,"Value":114668119,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":258325,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1294783,"Executor Run Time":0,"Executor CPU Time":952332,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":310,"Index":108,"Attempt":0,"Partition ID":108,"Launch Time":1714515296615,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":305,"Index":103,"Attempt":0,"Partition ID":103,"Launch Time":1714515296612,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296615,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":91,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1065904,"Value":127896185,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":881835,"Value":115549954,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":260795,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1065904,"Executor Run Time":0,"Executor CPU Time":881835,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":311,"Index":109,"Attempt":0,"Partition ID":109,"Launch Time":1714515296615,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":306,"Index":104,"Attempt":0,"Partition ID":104,"Launch Time":1714515296613,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296616,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":899374,"Value":128795559,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":708323,"Value":116258277,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":263222,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":899374,"Executor Run Time":0,"Executor CPU Time":708323,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":312,"Index":110,"Attempt":0,"Partition ID":110,"Launch Time":1714515296616,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":308,"Index":106,"Attempt":0,"Partition ID":106,"Launch Time":1714515296614,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296616,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":848216,"Value":129643775,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":785050,"Value":117043327,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":265649,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":848216,"Executor Run Time":0,"Executor CPU Time":785050,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":313,"Index":111,"Attempt":0,"Partition ID":111,"Launch Time":1714515296616,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":307,"Index":105,"Attempt":0,"Partition ID":105,"Launch Time":1714515296613,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296616,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":92,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1108792,"Value":130752567,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":834198,"Value":117877525,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":268119,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1108792,"Executor Run Time":0,"Executor CPU Time":834198,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":314,"Index":112,"Attempt":0,"Partition ID":112,"Launch Time":1714515296617,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":309,"Index":107,"Attempt":0,"Partition ID":107,"Launch Time":1714515296614,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296617,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":93,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1053556,"Value":131806123,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":919862,"Value":118797387,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":270589,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1053556,"Executor Run Time":0,"Executor CPU Time":919862,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":315,"Index":113,"Attempt":0,"Partition ID":113,"Launch Time":1714515296617,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":310,"Index":108,"Attempt":0,"Partition ID":108,"Launch Time":1714515296615,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296617,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":855826,"Value":132661949,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":662566,"Value":119459953,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":273016,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":855826,"Executor Run Time":0,"Executor CPU Time":662566,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":316,"Index":114,"Attempt":0,"Partition ID":114,"Launch Time":1714515296618,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":317,"Index":115,"Attempt":0,"Partition ID":115,"Launch Time":1714515296618,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":311,"Index":109,"Attempt":0,"Partition ID":109,"Launch Time":1714515296615,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296618,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":94,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1010219,"Value":133672168,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":903353,"Value":120363306,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":275486,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1010219,"Executor Run Time":0,"Executor CPU Time":903353,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":318,"Index":116,"Attempt":0,"Partition ID":116,"Launch Time":1714515296618,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":313,"Index":111,"Attempt":0,"Partition ID":111,"Launch Time":1714515296616,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296618,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":823908,"Value":134496076,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":657786,"Value":121021092,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":277913,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":823908,"Executor Run Time":0,"Executor CPU Time":657786,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":312,"Index":110,"Attempt":0,"Partition ID":110,"Launch Time":1714515296616,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296619,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1082593,"Value":135578669,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":806220,"Value":121827312,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":280383,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1082593,"Executor Run Time":0,"Executor CPU Time":806220,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":319,"Index":117,"Attempt":0,"Partition ID":117,"Launch Time":1714515296619,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":320,"Index":118,"Attempt":0,"Partition ID":118,"Launch Time":1714515296619,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":315,"Index":113,"Attempt":0,"Partition ID":113,"Launch Time":1714515296617,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296619,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":660746,"Value":136239415,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":618248,"Value":122445560,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":282810,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":660746,"Executor Run Time":0,"Executor CPU Time":618248,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":314,"Index":112,"Attempt":0,"Partition ID":112,"Launch Time":1714515296617,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296619,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":947072,"Value":137186487,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":908213,"Value":123353773,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":285237,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":947072,"Executor Run Time":0,"Executor CPU Time":908213,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":321,"Index":119,"Attempt":0,"Partition ID":119,"Launch Time":1714515296621,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":316,"Index":114,"Attempt":0,"Partition ID":114,"Launch Time":1714515296618,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296621,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":96,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1249106,"Value":138435593,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":78,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1094103,"Value":124447876,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":287750,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1249106,"Executor Run Time":1,"Executor CPU Time":1094103,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":322,"Index":120,"Attempt":0,"Partition ID":120,"Launch Time":1714515296621,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":323,"Index":121,"Attempt":0,"Partition ID":121,"Launch Time":1714515296622,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":317,"Index":115,"Attempt":0,"Partition ID":115,"Launch Time":1714515296618,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296622,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":97,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1370820,"Value":139806413,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":79,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1028997,"Value":125476873,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":290263,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1370820,"Executor Run Time":1,"Executor CPU Time":1028997,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":318,"Index":116,"Attempt":0,"Partition ID":116,"Launch Time":1714515296618,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296622,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":98,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1360289,"Value":141166702,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":80,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1065886,"Value":126542759,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":292776,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1360289,"Executor Run Time":1,"Executor CPU Time":1065886,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":324,"Index":122,"Attempt":0,"Partition ID":122,"Launch Time":1714515296622,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":325,"Index":123,"Attempt":0,"Partition ID":123,"Launch Time":1714515296622,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":320,"Index":118,"Attempt":0,"Partition ID":118,"Launch Time":1714515296619,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296622,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":846277,"Value":142012979,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":872225,"Value":127414984,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":295203,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":846277,"Executor Run Time":0,"Executor CPU Time":872225,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":319,"Index":117,"Attempt":0,"Partition ID":117,"Launch Time":1714515296619,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296622,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":99,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1214218,"Value":143227197,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":81,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1031987,"Value":128446971,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":297716,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1214218,"Executor Run Time":1,"Executor CPU Time":1031987,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":326,"Index":124,"Attempt":0,"Partition ID":124,"Launch Time":1714515296624,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":325,"Index":123,"Attempt":0,"Partition ID":123,"Launch Time":1714515296622,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296624,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":870615,"Value":144097812,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":749902,"Value":129196873,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":300143,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":870615,"Executor Run Time":0,"Executor CPU Time":749902,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":327,"Index":125,"Attempt":0,"Partition ID":125,"Launch Time":1714515296625,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":328,"Index":126,"Attempt":0,"Partition ID":126,"Launch Time":1714515296625,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":321,"Index":119,"Attempt":0,"Partition ID":119,"Launch Time":1714515296621,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296625,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":100,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1297673,"Value":145395485,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":82,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1018927,"Value":130215800,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":302656,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1297673,"Executor Run Time":1,"Executor CPU Time":1018927,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":322,"Index":120,"Attempt":0,"Partition ID":120,"Launch Time":1714515296621,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296625,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":101,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1265544,"Value":146661029,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":933733,"Value":131149533,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":305126,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1265544,"Executor Run Time":0,"Executor CPU Time":933733,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":329,"Index":127,"Attempt":0,"Partition ID":127,"Launch Time":1714515296625,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":330,"Index":128,"Attempt":0,"Partition ID":128,"Launch Time":1714515296626,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":323,"Index":121,"Attempt":0,"Partition ID":121,"Launch Time":1714515296622,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296626,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":102,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1213718,"Value":147874747,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":930262,"Value":132079795,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":307596,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1213718,"Executor Run Time":0,"Executor CPU Time":930262,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":324,"Index":122,"Attempt":0,"Partition ID":122,"Launch Time":1714515296622,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296626,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":103,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1056405,"Value":148931152,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":901954,"Value":132981749,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":310066,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1056405,"Executor Run Time":0,"Executor CPU Time":901954,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":331,"Index":129,"Attempt":0,"Partition ID":129,"Launch Time":1714515296627,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":327,"Index":125,"Attempt":0,"Partition ID":125,"Launch Time":1714515296625,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296628,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":104,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1016577,"Value":149947729,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":871456,"Value":133853205,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":312536,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1016577,"Executor Run Time":0,"Executor CPU Time":871456,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":332,"Index":130,"Attempt":0,"Partition ID":130,"Launch Time":1714515296628,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":326,"Index":124,"Attempt":0,"Partition ID":124,"Launch Time":1714515296624,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296628,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":105,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1197408,"Value":151145137,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":83,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1176021,"Value":135029226,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":315049,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1197408,"Executor Run Time":1,"Executor CPU Time":1176021,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":333,"Index":131,"Attempt":0,"Partition ID":131,"Launch Time":1714515296629,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":328,"Index":126,"Attempt":0,"Partition ID":126,"Launch Time":1714515296625,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296629,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":106,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1208277,"Value":152353414,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1032517,"Value":136061743,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":317562,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1208277,"Executor Run Time":1,"Executor CPU Time":1032517,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":334,"Index":132,"Attempt":0,"Partition ID":132,"Launch Time":1714515296629,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":335,"Index":133,"Attempt":0,"Partition ID":133,"Launch Time":1714515296629,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":330,"Index":128,"Attempt":0,"Partition ID":128,"Launch Time":1714515296626,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296629,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":833997,"Value":153187411,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":944841,"Value":137006584,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":319989,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":833997,"Executor Run Time":0,"Executor CPU Time":944841,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":329,"Index":127,"Attempt":0,"Partition ID":127,"Launch Time":1714515296625,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296629,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":107,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1108122,"Value":154295533,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":927793,"Value":137934377,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":322459,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1108122,"Executor Run Time":0,"Executor CPU Time":927793,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":336,"Index":134,"Attempt":0,"Partition ID":134,"Launch Time":1714515296630,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":331,"Index":129,"Attempt":0,"Partition ID":129,"Launch Time":1714515296627,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296631,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":108,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1019318,"Value":155314851,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":966970,"Value":138901347,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":324929,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1019318,"Executor Run Time":0,"Executor CPU Time":966970,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":337,"Index":135,"Attempt":0,"Partition ID":135,"Launch Time":1714515296631,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":332,"Index":130,"Attempt":0,"Partition ID":130,"Launch Time":1714515296628,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296631,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":109,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1148570,"Value":156463421,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":906804,"Value":139808151,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":327399,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1148570,"Executor Run Time":0,"Executor CPU Time":906804,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":338,"Index":136,"Attempt":0,"Partition ID":136,"Launch Time":1714515296631,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":333,"Index":131,"Attempt":0,"Partition ID":131,"Launch Time":1714515296629,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296632,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":110,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1177469,"Value":157640890,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":85,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1057506,"Value":140865657,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":329912,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1177469,"Executor Run Time":1,"Executor CPU Time":1057506,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":339,"Index":137,"Attempt":0,"Partition ID":137,"Launch Time":1714515296632,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":335,"Index":133,"Attempt":0,"Partition ID":133,"Launch Time":1714515296629,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296632,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":111,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1114512,"Value":158755402,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":927994,"Value":141793651,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":332382,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1114512,"Executor Run Time":0,"Executor CPU Time":927994,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":340,"Index":138,"Attempt":0,"Partition ID":138,"Launch Time":1714515296632,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":334,"Index":132,"Attempt":0,"Partition ID":132,"Launch Time":1714515296629,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296632,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":112,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1283324,"Value":160038726,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":987019,"Value":142780670,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":334852,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1283324,"Executor Run Time":0,"Executor CPU Time":987019,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":341,"Index":139,"Attempt":0,"Partition ID":139,"Launch Time":1714515296633,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":336,"Index":134,"Attempt":0,"Partition ID":134,"Launch Time":1714515296630,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296633,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":113,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":999929,"Value":161038655,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":866765,"Value":143647435,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":337322,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":999929,"Executor Run Time":0,"Executor CPU Time":866765,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":342,"Index":140,"Attempt":0,"Partition ID":140,"Launch Time":1714515296633,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":337,"Index":135,"Attempt":0,"Partition ID":135,"Launch Time":1714515296631,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296634,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":114,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1116313,"Value":162154968,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":813188,"Value":144460623,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":339792,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1116313,"Executor Run Time":0,"Executor CPU Time":813188,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":343,"Index":141,"Attempt":0,"Partition ID":141,"Launch Time":1714515296634,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":338,"Index":136,"Attempt":0,"Partition ID":136,"Launch Time":1714515296631,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296634,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":937813,"Value":163092781,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":816177,"Value":145276800,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":342219,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":937813,"Executor Run Time":0,"Executor CPU Time":816177,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":344,"Index":142,"Attempt":0,"Partition ID":142,"Launch Time":1714515296634,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":339,"Index":137,"Attempt":0,"Partition ID":137,"Launch Time":1714515296632,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296634,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":115,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1042296,"Value":164135077,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":830498,"Value":146107298,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":344689,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1042296,"Executor Run Time":0,"Executor CPU Time":830498,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":345,"Index":143,"Attempt":0,"Partition ID":143,"Launch Time":1714515296635,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":346,"Index":144,"Attempt":0,"Partition ID":144,"Launch Time":1714515296635,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":340,"Index":138,"Attempt":0,"Partition ID":138,"Launch Time":1714515296632,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296635,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":116,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1137882,"Value":165272959,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":917782,"Value":147025080,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":347159,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1137882,"Executor Run Time":0,"Executor CPU Time":917782,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":341,"Index":139,"Attempt":0,"Partition ID":139,"Launch Time":1714515296633,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296635,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":825428,"Value":166098387,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":672485,"Value":147697565,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":349586,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":825428,"Executor Run Time":0,"Executor CPU Time":672485,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":347,"Index":145,"Attempt":0,"Partition ID":145,"Launch Time":1714515296636,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":342,"Index":140,"Attempt":0,"Partition ID":140,"Launch Time":1714515296633,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296636,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":117,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1080694,"Value":167179081,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":914273,"Value":148611838,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":352056,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1080694,"Executor Run Time":0,"Executor CPU Time":914273,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":348,"Index":146,"Attempt":0,"Partition ID":146,"Launch Time":1714515296636,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":343,"Index":141,"Attempt":0,"Partition ID":141,"Launch Time":1714515296634,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296637,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":993259,"Value":168172340,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":941661,"Value":149553499,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":354483,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":993259,"Executor Run Time":0,"Executor CPU Time":941661,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":349,"Index":147,"Attempt":0,"Partition ID":147,"Launch Time":1714515296637,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":344,"Index":142,"Attempt":0,"Partition ID":142,"Launch Time":1714515296634,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296637,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":118,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1002857,"Value":169175197,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":996500,"Value":150549999,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":356953,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1002857,"Executor Run Time":0,"Executor CPU Time":996500,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":350,"Index":148,"Attempt":0,"Partition ID":148,"Launch Time":1714515296637,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":345,"Index":143,"Attempt":0,"Partition ID":143,"Launch Time":1714515296635,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296637,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":119,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1082254,"Value":170257451,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":666696,"Value":151216695,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":359423,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1082254,"Executor Run Time":0,"Executor CPU Time":666696,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":351,"Index":149,"Attempt":0,"Partition ID":149,"Launch Time":1714515296638,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":346,"Index":144,"Attempt":0,"Partition ID":144,"Launch Time":1714515296635,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296638,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":120,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1116272,"Value":171373723,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":86,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1053936,"Value":152270631,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":361936,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1116272,"Executor Run Time":1,"Executor CPU Time":1053936,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":352,"Index":150,"Attempt":0,"Partition ID":150,"Launch Time":1714515296638,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":347,"Index":145,"Attempt":0,"Partition ID":145,"Launch Time":1714515296636,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296639,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":121,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1109873,"Value":172483596,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":780540,"Value":153051171,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":364406,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1109873,"Executor Run Time":0,"Executor CPU Time":780540,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":353,"Index":151,"Attempt":0,"Partition ID":151,"Launch Time":1714515296639,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":349,"Index":147,"Attempt":0,"Partition ID":147,"Launch Time":1714515296637,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296639,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":813918,"Value":173297514,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":702504,"Value":153753675,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":366833,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":813918,"Executor Run Time":0,"Executor CPU Time":702504,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":354,"Index":152,"Attempt":0,"Partition ID":152,"Launch Time":1714515296639,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":355,"Index":153,"Attempt":0,"Partition ID":153,"Launch Time":1714515296639,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":348,"Index":146,"Attempt":0,"Partition ID":146,"Launch Time":1714515296636,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296640,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":122,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1059936,"Value":174357450,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":806688,"Value":154560363,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":369303,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1059936,"Executor Run Time":0,"Executor CPU Time":806688,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":350,"Index":148,"Attempt":0,"Partition ID":148,"Launch Time":1714515296637,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296640,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":878315,"Value":175235765,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":722403,"Value":155282766,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":371730,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":878315,"Executor Run Time":0,"Executor CPU Time":722403,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":356,"Index":154,"Attempt":0,"Partition ID":154,"Launch Time":1714515296640,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":351,"Index":149,"Attempt":0,"Partition ID":149,"Launch Time":1714515296638,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296640,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":979169,"Value":176214934,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":87,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1061456,"Value":156344222,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":374200,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":979169,"Executor Run Time":1,"Executor CPU Time":1061456,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":357,"Index":155,"Attempt":0,"Partition ID":155,"Launch Time":1714515296641,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":352,"Index":150,"Attempt":0,"Partition ID":150,"Launch Time":1714515296638,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296641,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":821827,"Value":177036761,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":707104,"Value":157051326,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":376627,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":821827,"Executor Run Time":0,"Executor CPU Time":707104,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":358,"Index":156,"Attempt":0,"Partition ID":156,"Launch Time":1714515296642,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":353,"Index":151,"Attempt":0,"Partition ID":151,"Launch Time":1714515296639,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296642,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":123,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1275475,"Value":178312236,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":874274,"Value":157925600,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":379097,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1275475,"Executor Run Time":0,"Executor CPU Time":874274,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":359,"Index":157,"Attempt":0,"Partition ID":157,"Launch Time":1714515296642,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":354,"Index":152,"Attempt":0,"Partition ID":152,"Launch Time":1714515296639,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296642,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":124,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1118153,"Value":179430389,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":744432,"Value":158670032,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":381567,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1118153,"Executor Run Time":0,"Executor CPU Time":744432,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":360,"Index":158,"Attempt":0,"Partition ID":158,"Launch Time":1714515296642,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":361,"Index":159,"Attempt":0,"Partition ID":159,"Launch Time":1714515296643,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":355,"Index":153,"Attempt":0,"Partition ID":153,"Launch Time":1714515296639,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296643,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":125,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1052726,"Value":180483115,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":788489,"Value":159458521,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":384037,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1052726,"Executor Run Time":0,"Executor CPU Time":788489,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":356,"Index":154,"Attempt":0,"Partition ID":154,"Launch Time":1714515296640,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296643,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":798648,"Value":181281763,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":669247,"Value":160127768,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":386464,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":798648,"Executor Run Time":0,"Executor CPU Time":669247,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":362,"Index":160,"Attempt":0,"Partition ID":160,"Launch Time":1714515296643,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":357,"Index":155,"Attempt":0,"Partition ID":155,"Launch Time":1714515296641,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296643,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":126,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1016397,"Value":182298160,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":886765,"Value":161014533,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":388934,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1016397,"Executor Run Time":0,"Executor CPU Time":886765,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":363,"Index":161,"Attempt":0,"Partition ID":161,"Launch Time":1714515296644,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":359,"Index":157,"Attempt":0,"Partition ID":157,"Launch Time":1714515296642,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296644,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":831957,"Value":183130117,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":647297,"Value":161661830,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":391361,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":831957,"Executor Run Time":0,"Executor CPU Time":647297,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":364,"Index":162,"Attempt":0,"Partition ID":162,"Launch Time":1714515296644,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":358,"Index":156,"Attempt":0,"Partition ID":156,"Launch Time":1714515296642,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296644,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":127,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1089044,"Value":184219161,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":820997,"Value":162482827,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":393831,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1089044,"Executor Run Time":0,"Executor CPU Time":820997,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":365,"Index":163,"Attempt":0,"Partition ID":163,"Launch Time":1714515296645,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":360,"Index":158,"Attempt":0,"Partition ID":158,"Launch Time":1714515296642,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296645,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":936631,"Value":185155792,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":766911,"Value":163249738,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":396258,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":936631,"Executor Run Time":0,"Executor CPU Time":766911,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":366,"Index":164,"Attempt":0,"Partition ID":164,"Launch Time":1714515296645,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":361,"Index":159,"Attempt":0,"Partition ID":159,"Launch Time":1714515296643,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296645,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":128,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1056995,"Value":186212787,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":788440,"Value":164038178,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":398728,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1056995,"Executor Run Time":0,"Executor CPU Time":788440,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":367,"Index":165,"Attempt":0,"Partition ID":165,"Launch Time":1714515296645,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":362,"Index":160,"Attempt":0,"Partition ID":160,"Launch Time":1714515296643,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296646,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":129,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1069465,"Value":187282252,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":884655,"Value":164922833,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":401198,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1069465,"Executor Run Time":0,"Executor CPU Time":884655,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":368,"Index":166,"Attempt":0,"Partition ID":166,"Launch Time":1714515296647,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":365,"Index":163,"Attempt":0,"Partition ID":163,"Launch Time":1714515296645,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296647,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":942702,"Value":188224954,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":595599,"Value":165518432,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":403625,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":942702,"Executor Run Time":0,"Executor CPU Time":595599,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":369,"Index":167,"Attempt":0,"Partition ID":167,"Launch Time":1714515296647,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":363,"Index":161,"Attempt":0,"Partition ID":161,"Launch Time":1714515296644,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296647,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":130,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1180340,"Value":189405294,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":925882,"Value":166444314,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":406095,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1180340,"Executor Run Time":0,"Executor CPU Time":925882,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":370,"Index":168,"Attempt":0,"Partition ID":168,"Launch Time":1714515296647,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":364,"Index":162,"Attempt":0,"Partition ID":162,"Launch Time":1714515296644,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296647,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":131,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1274664,"Value":190679958,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":644057,"Value":167088371,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":408565,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1274664,"Executor Run Time":0,"Executor CPU Time":644057,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":371,"Index":169,"Attempt":0,"Partition ID":169,"Launch Time":1714515296647,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":366,"Index":164,"Attempt":0,"Partition ID":164,"Launch Time":1714515296645,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296648,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":132,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1040697,"Value":191720655,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":847436,"Value":167935807,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":411035,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1040697,"Executor Run Time":0,"Executor CPU Time":847436,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":372,"Index":170,"Attempt":0,"Partition ID":170,"Launch Time":1714515296648,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":367,"Index":165,"Attempt":0,"Partition ID":165,"Launch Time":1714515296645,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296648,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":133,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1045746,"Value":192766401,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":888154,"Value":168823961,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":413505,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1045746,"Executor Run Time":0,"Executor CPU Time":888154,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":373,"Index":171,"Attempt":0,"Partition ID":171,"Launch Time":1714515296649,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":368,"Index":166,"Attempt":0,"Partition ID":166,"Launch Time":1714515296647,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296653,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":134,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1044276,"Value":193810677,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":731954,"Value":169555915,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":415975,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1044276,"Executor Run Time":0,"Executor CPU Time":731954,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":374,"Index":172,"Attempt":0,"Partition ID":172,"Launch Time":1714515296654,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":371,"Index":169,"Attempt":0,"Partition ID":169,"Launch Time":1714515296647,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296654,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":794000,"Value":194604677,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":4,"Value":91,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1162658,"Value":170718573,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":418488,"Internal":true,"Count Failed Values":true},{"ID":109,"Name":"internal.metrics.jvmGCTime","Update":3,"Value":15,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":794000,"Executor Run Time":4,"Executor CPU Time":1162658,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":3,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":375,"Index":173,"Attempt":0,"Partition ID":173,"Launch Time":1714515296654,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":369,"Index":167,"Attempt":0,"Partition ID":167,"Launch Time":1714515296647,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296654,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":135,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1042487,"Value":195647164,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":4,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1250486,"Value":171969059,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2556,"Value":421044,"Internal":true,"Count Failed Values":true},{"ID":109,"Name":"internal.metrics.jvmGCTime","Update":3,"Value":18,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1042487,"Executor Run Time":4,"Executor CPU Time":1250486,"Peak Execution Memory":0,"Result Size":2556,"JVM GC Time":3,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":376,"Index":174,"Attempt":0,"Partition ID":174,"Launch Time":1714515296655,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":377,"Index":175,"Attempt":0,"Partition ID":175,"Launch Time":1714515296655,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":370,"Index":168,"Attempt":0,"Partition ID":168,"Launch Time":1714515296647,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296655,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":967631,"Value":196614795,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":4,"Value":99,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1293603,"Value":173262662,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":423557,"Internal":true,"Count Failed Values":true},{"ID":109,"Name":"internal.metrics.jvmGCTime","Update":3,"Value":21,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":967631,"Executor Run Time":4,"Executor CPU Time":1293603,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":3,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":372,"Index":170,"Attempt":0,"Partition ID":170,"Launch Time":1714515296648,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296655,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":139,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1047876,"Value":197662671,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":100,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1260806,"Value":174523468,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2556,"Value":426113,"Internal":true,"Count Failed Values":true},{"ID":109,"Name":"internal.metrics.jvmGCTime","Update":3,"Value":24,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":1047876,"Executor Run Time":1,"Executor CPU Time":1260806,"Peak Execution Memory":0,"Result Size":2556,"JVM GC Time":3,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":378,"Index":176,"Attempt":0,"Partition ID":176,"Launch Time":1714515296656,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":373,"Index":171,"Attempt":0,"Partition ID":171,"Launch Time":1714515296649,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296656,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":140,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1291093,"Value":198953764,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":101,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1063115,"Value":175586583,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":428626,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1291093,"Executor Run Time":1,"Executor CPU Time":1063115,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":379,"Index":177,"Attempt":0,"Partition ID":177,"Launch Time":1714515296657,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":380,"Index":178,"Attempt":0,"Partition ID":178,"Launch Time":1714515296657,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":375,"Index":173,"Attempt":0,"Partition ID":173,"Launch Time":1714515296654,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296657,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":141,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1220957,"Value":200174721,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":888894,"Value":176475477,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":431096,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1220957,"Executor Run Time":0,"Executor CPU Time":888894,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":374,"Index":172,"Attempt":0,"Partition ID":172,"Launch Time":1714515296654,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296657,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":142,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1451756,"Value":201626477,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":102,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1156590,"Value":177632067,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":433609,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1451756,"Executor Run Time":1,"Executor CPU Time":1156590,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":381,"Index":179,"Attempt":0,"Partition ID":179,"Launch Time":1714515296658,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":382,"Index":180,"Attempt":0,"Partition ID":180,"Launch Time":1714515296658,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":376,"Index":174,"Attempt":0,"Partition ID":174,"Launch Time":1714515296655,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296658,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":143,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1214527,"Value":202841004,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":103,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1098473,"Value":178730540,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":436122,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1214527,"Executor Run Time":1,"Executor CPU Time":1098473,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":377,"Index":175,"Attempt":0,"Partition ID":175,"Launch Time":1714515296655,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296658,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":144,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1065775,"Value":203906779,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":983989,"Value":179714529,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":438592,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1065775,"Executor Run Time":0,"Executor CPU Time":983989,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":383,"Index":181,"Attempt":0,"Partition ID":181,"Launch Time":1714515296659,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":378,"Index":176,"Attempt":0,"Partition ID":176,"Launch Time":1714515296656,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296659,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":145,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1048867,"Value":204955646,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":104,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1011807,"Value":180726336,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":441105,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1048867,"Executor Run Time":1,"Executor CPU Time":1011807,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":384,"Index":182,"Attempt":0,"Partition ID":182,"Launch Time":1714515296660,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":379,"Index":177,"Attempt":0,"Partition ID":177,"Launch Time":1714515296657,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296660,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":146,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1108593,"Value":206064239,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":905743,"Value":181632079,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":443575,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1108593,"Executor Run Time":0,"Executor CPU Time":905743,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":385,"Index":183,"Attempt":0,"Partition ID":183,"Launch Time":1714515296660,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":386,"Index":184,"Attempt":0,"Partition ID":184,"Launch Time":1714515296660,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":380,"Index":178,"Attempt":0,"Partition ID":178,"Launch Time":1714515296657,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296660,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":918092,"Value":206982331,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":803718,"Value":182435797,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":446002,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":918092,"Executor Run Time":0,"Executor CPU Time":803718,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":381,"Index":179,"Attempt":0,"Partition ID":179,"Launch Time":1714515296658,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296660,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":763231,"Value":207745562,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":746812,"Value":183182609,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":448429,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":763231,"Executor Run Time":0,"Executor CPU Time":746812,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":387,"Index":185,"Attempt":0,"Partition ID":185,"Launch Time":1714515296661,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":382,"Index":180,"Attempt":0,"Partition ID":180,"Launch Time":1714515296658,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296661,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":147,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1084515,"Value":208830077,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":699924,"Value":183882533,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":450899,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1084515,"Executor Run Time":0,"Executor CPU Time":699924,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":388,"Index":186,"Attempt":0,"Partition ID":186,"Launch Time":1714515296662,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":384,"Index":182,"Attempt":0,"Partition ID":182,"Launch Time":1714515296660,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296662,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":770390,"Value":209600467,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":805540,"Value":184688073,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":453326,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":770390,"Executor Run Time":0,"Executor CPU Time":805540,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":389,"Index":187,"Attempt":0,"Partition ID":187,"Launch Time":1714515296662,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":383,"Index":181,"Attempt":0,"Partition ID":181,"Launch Time":1714515296659,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296663,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":148,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1247296,"Value":210847763,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":105,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1231056,"Value":185919129,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":455839,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1247296,"Executor Run Time":1,"Executor CPU Time":1231056,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":390,"Index":188,"Attempt":0,"Partition ID":188,"Launch Time":1714515296663,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":391,"Index":189,"Attempt":0,"Partition ID":189,"Launch Time":1714515296663,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":385,"Index":183,"Attempt":0,"Partition ID":183,"Launch Time":1714515296660,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296663,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":149,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1057216,"Value":211904979,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":884615,"Value":186803744,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":458309,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1057216,"Executor Run Time":0,"Executor CPU Time":884615,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":386,"Index":184,"Attempt":0,"Partition ID":184,"Launch Time":1714515296660,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296663,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":150,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":941901,"Value":212846880,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":772041,"Value":187575785,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":460779,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":941901,"Executor Run Time":0,"Executor CPU Time":772041,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":392,"Index":190,"Attempt":0,"Partition ID":190,"Launch Time":1714515296663,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":387,"Index":185,"Attempt":0,"Partition ID":185,"Launch Time":1714515296661,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296664,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":151,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1177020,"Value":214023900,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":763960,"Value":188339745,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":463249,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1177020,"Executor Run Time":0,"Executor CPU Time":763960,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":393,"Index":191,"Attempt":0,"Partition ID":191,"Launch Time":1714515296665,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":394,"Index":192,"Attempt":0,"Partition ID":192,"Launch Time":1714515296665,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":388,"Index":186,"Attempt":0,"Partition ID":186,"Launch Time":1714515296662,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296665,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":152,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1305423,"Value":215329323,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":106,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1065334,"Value":189405079,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":465762,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1305423,"Executor Run Time":1,"Executor CPU Time":1065334,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":389,"Index":187,"Attempt":0,"Partition ID":187,"Launch Time":1714515296662,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296666,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":153,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1197748,"Value":216527071,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":913833,"Value":190318912,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":468232,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1197748,"Executor Run Time":0,"Executor CPU Time":913833,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":395,"Index":193,"Attempt":0,"Partition ID":193,"Launch Time":1714515296666,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":396,"Index":194,"Attempt":0,"Partition ID":194,"Launch Time":1714515296666,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":390,"Index":188,"Attempt":0,"Partition ID":188,"Launch Time":1714515296663,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296666,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":977859,"Value":217504930,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":915783,"Value":191234695,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":470659,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":977859,"Executor Run Time":0,"Executor CPU Time":915783,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":397,"Index":195,"Attempt":0,"Partition ID":195,"Launch Time":1714515296666,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":391,"Index":189,"Attempt":0,"Partition ID":189,"Launch Time":1714515296663,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296666,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":952371,"Value":218457301,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":817617,"Value":192052312,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":473086,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":952371,"Executor Run Time":0,"Executor CPU Time":817617,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":392,"Index":190,"Attempt":0,"Partition ID":190,"Launch Time":1714515296663,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296666,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":785310,"Value":219242611,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":624477,"Value":192676789,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":475513,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":785310,"Executor Run Time":0,"Executor CPU Time":624477,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":398,"Index":196,"Attempt":0,"Partition ID":196,"Launch Time":1714515296668,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":394,"Index":192,"Attempt":0,"Partition ID":192,"Launch Time":1714515296665,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296668,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":154,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1051656,"Value":220294267,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":698475,"Value":193375264,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":477983,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1051656,"Executor Run Time":0,"Executor CPU Time":698475,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":399,"Index":197,"Attempt":0,"Partition ID":197,"Launch Time":1714515296668,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":393,"Index":191,"Attempt":0,"Partition ID":191,"Launch Time":1714515296665,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296668,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":155,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1060374,"Value":221354641,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":963152,"Value":194338416,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":480453,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1060374,"Executor Run Time":0,"Executor CPU Time":963152,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":400,"Index":198,"Attempt":0,"Partition ID":198,"Launch Time":1714515296669,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":401,"Index":199,"Attempt":0,"Partition ID":199,"Launch Time":1714515296669,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":396,"Index":194,"Attempt":0,"Partition ID":194,"Launch Time":1714515296666,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296669,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":805689,"Value":222160330,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":613399,"Value":194951815,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":482880,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":805689,"Executor Run Time":0,"Executor CPU Time":613399,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":395,"Index":193,"Attempt":0,"Partition ID":193,"Launch Time":1714515296666,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296669,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":919313,"Value":223079643,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":768551,"Value":195720366,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":485307,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":919313,"Executor Run Time":0,"Executor CPU Time":768551,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":397,"Index":195,"Attempt":0,"Partition ID":195,"Launch Time":1714515296666,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296670,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":850387,"Value":223930030,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":914062,"Value":196634428,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":487734,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":850387,"Executor Run Time":0,"Executor CPU Time":914062,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":398,"Index":196,"Attempt":0,"Partition ID":196,"Launch Time":1714515296668,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296671,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":156,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1225627,"Value":225155657,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":949091,"Value":197583519,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":490204,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1225627,"Executor Run Time":0,"Executor CPU Time":949091,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":399,"Index":197,"Attempt":0,"Partition ID":197,"Launch Time":1714515296668,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296671,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":901134,"Value":226056791,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":107,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":765619,"Value":198349138,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":492674,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":901134,"Executor Run Time":1,"Executor CPU Time":765619,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":401,"Index":199,"Attempt":0,"Partition ID":199,"Launch Time":1714515296669,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296672,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":157,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1216537,"Value":227273328,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":945311,"Value":199294449,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":495144,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1216537,"Executor Run Time":0,"Executor CPU Time":945311,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":400,"Index":198,"Attempt":0,"Partition ID":198,"Launch Time":1714515296669,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515296672,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":158,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1417017,"Value":228690345,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":108,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1079724,"Value":200374173,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":497657,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1417017,"Executor Run Time":1,"Executor CPU Time":1079724,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"processLine at CliDriver.java:336","Number of Tasks":200,"RDD Info":[{"RDD ID":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"13\",\"name\":\"Exchange\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"UNORDERED","Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[4],"Details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1714515296535,"Completion Time":1714515296672,"Accumulables":[{"ID":69,"Name":"local blocks read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":72,"Name":"local bytes read","Value":"84","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":73,"Name":"fetch wait time","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"records read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Value":158,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Value":228690345,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Value":108,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Value":200374173,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Value":497657,"Internal":true,"Count Failed Values":true},{"ID":109,"Name":"internal.metrics.jvmGCTime","Value":24,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":127,"Name":"internal.metrics.output.bytesWritten","Value":703,"Internal":true,"Count Failed Values":true},{"ID":128,"Name":"internal.metrics.output.recordsWritten","Value":1,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":3,"Completion Time":1714515296672,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":6,"time":1714515296752} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":7,"description":"\nINSERT INTO local.projects VALUES(\"krisnova\", \"aurae\")","details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nCommandResult (1)\n +- AppendData (6)\n +- AdaptiveSparkPlan (5)\n +- == Final Plan ==\n ShuffleQueryStage (4), Statistics(sizeInBytes=40.0 B, rowCount=1)\n +- Exchange (3)\n +- LocalTableScan (2)\n +- == Initial Plan ==\n Exchange (3)\n +- LocalTableScan (2)\n\n\n(1) CommandResult\nOutput: []\nArguments: \n\n(2) LocalTableScan\nOutput [2]: [creator#13, projectname#14]\nArguments: [creator#13, projectname#14]\n\n(3) Exchange\nInput [2]: [creator#13, projectname#14]\nArguments: hashpartitioning(creator#13, 200), REPARTITION_BY_NUM, [plan_id=42]\n\n(4) ShuffleQueryStage\nOutput [2]: [creator#13, projectname#14]\nArguments: 0\n\n(5) AdaptiveSparkPlan\nOutput [2]: [creator#13, projectname#14]\nArguments: isFinalPlan=true\n\n(6) AppendData\nInput [2]: [creator#13, projectname#14]\nArguments: org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$2253/0x0000775334f5b4c0@72e89ce2, IcebergWrite(table=local.projects, format=PARQUET)\n\n","sparkPlanInfo":{"nodeName":"CommandResult","simpleString":"CommandResult ","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":129,"metricType":"sum"}]},"time":1714515296757,"modifiedConfigs":{}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":7,"time":1714515296757} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":8,"description":"\nSELECT * FROM local.developers INNER JOIN local.projects ON local.projects.creator = local.developers.username","details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nAdaptiveSparkPlan (5)\n+- BroadcastHashJoin Inner BuildRight (4)\n :- BatchScan (1)\n +- BroadcastExchange (3)\n +- BatchScan (2)\n\n\n(1) BatchScan\nOutput [3]: [username#15, firstname#16, lastname#17]\nlocal.developers (branch=null) [filters=username IS NOT NULL, groupedBy=]\n\n(2) BatchScan\nOutput [2]: [creator#18, projectname#19]\nlocal.projects (branch=null) [filters=creator IS NOT NULL, groupedBy=]\n\n(3) BroadcastExchange\nInput [2]: [creator#18, projectname#19]\nArguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=66]\n\n(4) BroadcastHashJoin\nLeft keys [1]: [username#15]\nRight keys [1]: [creator#18]\nJoin condition: None\n\n(5) AdaptiveSparkPlan\nOutput [5]: [username#15, firstname#16, lastname#17, creator#18, projectname#19]\nArguments: isFinalPlan=false\n\n","sparkPlanInfo":{"nodeName":"AdaptiveSparkPlan","simpleString":"AdaptiveSparkPlan isFinalPlan=false","children":[{"nodeName":"BroadcastHashJoin","simpleString":"BroadcastHashJoin [username#15], [creator#18], Inner, BuildRight, false","children":[{"nodeName":"BatchScan","simpleString":"BatchScan[username#15, firstname#16, lastname#17] local.developers (branch=null) [filters=username IS NOT NULL, groupedBy=] RuntimeFilters: []","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":131,"metricType":"sum"},{"name":"number of file splits read","accumulatorId":132,"metricType":"v2Custom_org.apache.iceberg.spark.source.metrics.NumSplits"},{"name":"number of row deletes applied","accumulatorId":133,"metricType":"v2Custom_org.apache.iceberg.spark.source.metrics.NumDeletes"}]},{"nodeName":"BroadcastExchange","simpleString":"BroadcastExchange HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=66]","children":[{"nodeName":"BatchScan","simpleString":"BatchScan[creator#18, projectname#19] local.projects (branch=null) [filters=creator IS NOT NULL, groupedBy=] RuntimeFilters: []","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":139,"metricType":"sum"},{"name":"number of file splits read","accumulatorId":140,"metricType":"v2Custom_org.apache.iceberg.spark.source.metrics.NumSplits"},{"name":"number of row deletes applied","accumulatorId":141,"metricType":"v2Custom_org.apache.iceberg.spark.source.metrics.NumDeletes"}]}],"metadata":{},"metrics":[{"name":"time to broadcast","accumulatorId":138,"metricType":"timing"},{"name":"time to build","accumulatorId":137,"metricType":"timing"},{"name":"time to collect","accumulatorId":136,"metricType":"timing"},{"name":"number of output rows","accumulatorId":135,"metricType":"sum"},{"name":"data size","accumulatorId":134,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":130,"metricType":"sum"}]}],"metadata":{},"metrics":[]},"time":1714515297112,"modifiedConfigs":{}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate","executionId":8,"physicalPlanDescription":"== Physical Plan ==\nAdaptiveSparkPlan (9)\n+- == Current Plan ==\n BroadcastHashJoin Inner BuildRight (6)\n :- BatchScan (1)\n +- BroadcastQueryStage (5)\n +- BroadcastExchange (4)\n +- * ColumnarToRow (3)\n +- BatchScan (2)\n+- == Initial Plan ==\n BroadcastHashJoin Inner BuildRight (8)\n :- BatchScan (1)\n +- BroadcastExchange (7)\n +- BatchScan (2)\n\n\n(1) BatchScan\nOutput [3]: [username#15, firstname#16, lastname#17]\nlocal.developers (branch=null) [filters=username IS NOT NULL, groupedBy=]\n\n(2) BatchScan\nOutput [2]: [creator#18, projectname#19]\nlocal.projects (branch=null) [filters=creator IS NOT NULL, groupedBy=]\n\n(3) ColumnarToRow [codegen id : 1]\nInput [2]: [creator#18, projectname#19]\n\n(4) BroadcastExchange\nInput [2]: [creator#18, projectname#19]\nArguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=75]\n\n(5) BroadcastQueryStage\nOutput [2]: [creator#18, projectname#19]\nArguments: 0\n\n(6) BroadcastHashJoin\nLeft keys [1]: [username#15]\nRight keys [1]: [creator#18]\nJoin condition: None\n\n(7) BroadcastExchange\nInput [2]: [creator#18, projectname#19]\nArguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=66]\n\n(8) BroadcastHashJoin\nLeft keys [1]: [username#15]\nRight keys [1]: [creator#18]\nJoin condition: None\n\n(9) AdaptiveSparkPlan\nOutput [5]: [username#15, firstname#16, lastname#17, creator#18, projectname#19]\nArguments: isFinalPlan=false\n\n","sparkPlanInfo":{"nodeName":"AdaptiveSparkPlan","simpleString":"AdaptiveSparkPlan isFinalPlan=false","children":[{"nodeName":"BroadcastHashJoin","simpleString":"BroadcastHashJoin [username#15], [creator#18], Inner, BuildRight, false","children":[{"nodeName":"BatchScan","simpleString":"BatchScan[username#15, firstname#16, lastname#17] local.developers (branch=null) [filters=username IS NOT NULL, groupedBy=] RuntimeFilters: []","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":131,"metricType":"sum"},{"name":"number of file splits read","accumulatorId":132,"metricType":"v2Custom_org.apache.iceberg.spark.source.metrics.NumSplits"},{"name":"number of row deletes applied","accumulatorId":133,"metricType":"v2Custom_org.apache.iceberg.spark.source.metrics.NumDeletes"}]},{"nodeName":"BroadcastQueryStage","simpleString":"BroadcastQueryStage 0","children":[{"nodeName":"BroadcastExchange","simpleString":"BroadcastExchange HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=75]","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"ColumnarToRow","simpleString":"ColumnarToRow","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"BatchScan","simpleString":"BatchScan[creator#18, projectname#19] local.projects (branch=null) [filters=creator IS NOT NULL, groupedBy=] RuntimeFilters: []","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":139,"metricType":"sum"},{"name":"number of file splits read","accumulatorId":140,"metricType":"v2Custom_org.apache.iceberg.spark.source.metrics.NumSplits"},{"name":"number of row deletes applied","accumulatorId":141,"metricType":"v2Custom_org.apache.iceberg.spark.source.metrics.NumDeletes"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":149,"metricType":"sum"},{"name":"number of input batches","accumulatorId":150,"metricType":"sum"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":148,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"time to broadcast","accumulatorId":147,"metricType":"timing"},{"name":"time to build","accumulatorId":146,"metricType":"timing"},{"name":"time to collect","accumulatorId":145,"metricType":"timing"},{"name":"number of output rows","accumulatorId":144,"metricType":"sum"},{"name":"data size","accumulatorId":143,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":142,"metricType":"sum"}]}],"metadata":{},"metrics":[]}} +{"Event":"SparkListenerJobStart","Job ID":4,"Submission Time":1714515297166,"Stage Infos":[{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:264","Number of Tasks":1,"RDD Info":[{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"BroadcastExchange\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:264","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:264","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"22\",\"name\":\"BatchScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:264","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"DataSourceRDD","Scope":"{\"id\":\"22\",\"name\":\"BatchScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:264","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:191)\njava.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)\njava.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)\njava.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)\njava.base/java.lang.Thread.run(Thread.java:840)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[6],"Properties":{"spark.sql.warehouse.dir":"file:/home/holden/repos/sparklens/spark-warehouse","spark.executor.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.driver.host":"10.69.200.180","spark.eventLog.enabled":"true","spark.job.interruptOnCancel":"true","spark.driver.port":"37317","__fetch_continuous_blocks_in_batch_enabled":"true","spark.sql.catalog.local.type":"hadoop","spark.jars":"","spark.sql.catalog.spark_catalog":"org.apache.iceberg.spark.SparkSessionCatalog","spark.sql.datetime.java8API.enabled":"true","spark.sql.hive.version":"2.3.9","spark.app.name":"fresh","spark.rdd.scope":"{\"id\":\"18\",\"name\":\"BroadcastExchange\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.submitTime":"1714515291527","spark.job.description":"broadcast exchange (runId 0cb43d56-21c6-4e07-88d0-b3ad3e90b4c4)","spark.app.startTime":"1714515291883","spark.executor.id":"driver","spark.sql.catalog.local.warehouse":"/home/holden/repos/sparklens/warehouse","spark.jobGroup.id":"0cb43d56-21c6-4e07-88d0-b3ad3e90b4c4","spark.driver.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.sql.catalog.local":"org.apache.iceberg.spark.SparkCatalog","spark.submit.deployMode":"client","spark.master":"local[5]","spark.sql.execution.id":"8","spark.sql.catalogImplementation":"hive","spark.sql.catalog.spark_catalog.type":"hive","spark.app.id":"local-1714515292260"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:264","Number of Tasks":1,"RDD Info":[{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"BroadcastExchange\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:264","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:264","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"22\",\"name\":\"BatchScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:264","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"DataSourceRDD","Scope":"{\"id\":\"22\",\"name\":\"BatchScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:264","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:191)\njava.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)\njava.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)\njava.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)\njava.base/java.lang.Thread.run(Thread.java:840)","Submission Time":1714515297167,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/holden/repos/sparklens/spark-warehouse","spark.executor.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.driver.host":"10.69.200.180","spark.eventLog.enabled":"true","spark.job.interruptOnCancel":"true","spark.driver.port":"37317","__fetch_continuous_blocks_in_batch_enabled":"true","spark.sql.catalog.local.type":"hadoop","spark.jars":"","spark.sql.catalog.spark_catalog":"org.apache.iceberg.spark.SparkSessionCatalog","spark.sql.datetime.java8API.enabled":"true","spark.sql.hive.version":"2.3.9","spark.app.name":"fresh","spark.rdd.scope":"{\"id\":\"18\",\"name\":\"BroadcastExchange\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.submitTime":"1714515291527","spark.job.description":"broadcast exchange (runId 0cb43d56-21c6-4e07-88d0-b3ad3e90b4c4)","spark.app.startTime":"1714515291883","spark.executor.id":"driver","spark.sql.catalog.local.warehouse":"/home/holden/repos/sparklens/warehouse","spark.jobGroup.id":"0cb43d56-21c6-4e07-88d0-b3ad3e90b4c4","spark.driver.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.sql.catalog.local":"org.apache.iceberg.spark.SparkCatalog","spark.submit.deployMode":"client","spark.master":"local[5]","spark.sql.execution.id":"8","spark.sql.catalogImplementation":"hive","spark.sql.catalog.spark_catalog.type":"hive","spark.app.id":"local-1714515292260"}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":402,"Index":0,"Attempt":0,"Partition ID":0,"Launch Time":1714515297169,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":402,"Index":0,"Attempt":0,"Partition ID":0,"Launch Time":1714515297169,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515297361,"Failed":false,"Killed":false,"Accumulables":[{"ID":139,"Name":"number of output rows","Update":"30","Value":"30","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":140,"Name":"number of file splits read","Update":"30","Value":"30","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":148,"Name":"duration","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":149,"Name":"number of output rows","Update":"30","Value":"30","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":150,"Name":"number of input batches","Update":"30","Value":"30","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":151,"Name":"internal.metrics.executorDeserializeTime","Update":15,"Value":15,"Internal":true,"Count Failed Values":true},{"ID":152,"Name":"internal.metrics.executorDeserializeCpuTime","Update":15157208,"Value":15157208,"Internal":true,"Count Failed Values":true},{"ID":153,"Name":"internal.metrics.executorRunTime","Update":170,"Value":170,"Internal":true,"Count Failed Values":true},{"ID":154,"Name":"internal.metrics.executorCpuTime","Update":168722285,"Value":168722285,"Internal":true,"Count Failed Values":true},{"ID":155,"Name":"internal.metrics.resultSize","Update":1958,"Value":1958,"Internal":true,"Count Failed Values":true},{"ID":172,"Name":"internal.metrics.input.bytesRead","Update":48870,"Value":48870,"Internal":true,"Count Failed Values":true},{"ID":173,"Name":"internal.metrics.input.recordsRead","Update":30,"Value":30,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":15,"Executor Deserialize CPU Time":15157208,"Executor Run Time":170,"Executor CPU Time":168722285,"Peak Execution Memory":0,"Result Size":1958,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":48870,"Records Read":30},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:264","Number of Tasks":1,"RDD Info":[{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"BroadcastExchange\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:264","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:264","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"22\",\"name\":\"BatchScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:264","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"DataSourceRDD","Scope":"{\"id\":\"22\",\"name\":\"BatchScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:264","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:191)\njava.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)\njava.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)\njava.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)\njava.base/java.lang.Thread.run(Thread.java:840)","Submission Time":1714515297167,"Completion Time":1714515297362,"Accumulables":[{"ID":139,"Name":"number of output rows","Value":"30","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":140,"Name":"number of file splits read","Value":"30","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":148,"Name":"duration","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":149,"Name":"number of output rows","Value":"30","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":150,"Name":"number of input batches","Value":"30","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":151,"Name":"internal.metrics.executorDeserializeTime","Value":15,"Internal":true,"Count Failed Values":true},{"ID":152,"Name":"internal.metrics.executorDeserializeCpuTime","Value":15157208,"Internal":true,"Count Failed Values":true},{"ID":153,"Name":"internal.metrics.executorRunTime","Value":170,"Internal":true,"Count Failed Values":true},{"ID":154,"Name":"internal.metrics.executorCpuTime","Value":168722285,"Internal":true,"Count Failed Values":true},{"ID":155,"Name":"internal.metrics.resultSize","Value":1958,"Internal":true,"Count Failed Values":true},{"ID":172,"Name":"internal.metrics.input.bytesRead","Value":48870,"Internal":true,"Count Failed Values":true},{"ID":173,"Name":"internal.metrics.input.recordsRead","Value":30,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":4,"Completion Time":1714515297362,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":8,"accumUpdates":[[147,7],[146,13],[145,240],[144,30],[143,4195312]]} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate","executionId":8,"physicalPlanDescription":"== Physical Plan ==\nAdaptiveSparkPlan (10)\n+- == Final Plan ==\n * BroadcastHashJoin Inner BuildRight (7)\n :- * ColumnarToRow (2)\n : +- BatchScan (1)\n +- BroadcastQueryStage (6), Statistics(sizeInBytes=4.0 MiB, rowCount=30)\n +- BroadcastExchange (5)\n +- * ColumnarToRow (4)\n +- BatchScan (3)\n+- == Initial Plan ==\n BroadcastHashJoin Inner BuildRight (9)\n :- BatchScan (1)\n +- BroadcastExchange (8)\n +- BatchScan (3)\n\n\n(1) BatchScan\nOutput [3]: [username#15, firstname#16, lastname#17]\nlocal.developers (branch=null) [filters=username IS NOT NULL, groupedBy=]\n\n(2) ColumnarToRow [codegen id : 2]\nInput [3]: [username#15, firstname#16, lastname#17]\n\n(3) BatchScan\nOutput [2]: [creator#18, projectname#19]\nlocal.projects (branch=null) [filters=creator IS NOT NULL, groupedBy=]\n\n(4) ColumnarToRow [codegen id : 1]\nInput [2]: [creator#18, projectname#19]\n\n(5) BroadcastExchange\nInput [2]: [creator#18, projectname#19]\nArguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=75]\n\n(6) BroadcastQueryStage\nOutput [2]: [creator#18, projectname#19]\nArguments: 0\n\n(7) BroadcastHashJoin [codegen id : 2]\nLeft keys [1]: [username#15]\nRight keys [1]: [creator#18]\nJoin condition: None\n\n(8) BroadcastExchange\nInput [2]: [creator#18, projectname#19]\nArguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=66]\n\n(9) BroadcastHashJoin\nLeft keys [1]: [username#15]\nRight keys [1]: [creator#18]\nJoin condition: None\n\n(10) AdaptiveSparkPlan\nOutput [5]: [username#15, firstname#16, lastname#17, creator#18, projectname#19]\nArguments: isFinalPlan=true\n\n","sparkPlanInfo":{"nodeName":"AdaptiveSparkPlan","simpleString":"AdaptiveSparkPlan isFinalPlan=true","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"BroadcastHashJoin","simpleString":"BroadcastHashJoin [username#15], [creator#18], Inner, BuildRight, false","children":[{"nodeName":"ColumnarToRow","simpleString":"ColumnarToRow","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"BatchScan","simpleString":"BatchScan[username#15, firstname#16, lastname#17] local.developers (branch=null) [filters=username IS NOT NULL, groupedBy=] RuntimeFilters: []","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":131,"metricType":"sum"},{"name":"number of file splits read","accumulatorId":132,"metricType":"v2Custom_org.apache.iceberg.spark.source.metrics.NumSplits"},{"name":"number of row deletes applied","accumulatorId":133,"metricType":"v2Custom_org.apache.iceberg.spark.source.metrics.NumDeletes"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":178,"metricType":"sum"},{"name":"number of input batches","accumulatorId":179,"metricType":"sum"}]},{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"BroadcastQueryStage","simpleString":"BroadcastQueryStage 0","children":[{"nodeName":"BroadcastExchange","simpleString":"BroadcastExchange HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=75]","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"ColumnarToRow","simpleString":"ColumnarToRow","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"BatchScan","simpleString":"BatchScan[creator#18, projectname#19] local.projects (branch=null) [filters=creator IS NOT NULL, groupedBy=] RuntimeFilters: []","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":139,"metricType":"sum"},{"name":"number of file splits read","accumulatorId":140,"metricType":"v2Custom_org.apache.iceberg.spark.source.metrics.NumSplits"},{"name":"number of row deletes applied","accumulatorId":141,"metricType":"v2Custom_org.apache.iceberg.spark.source.metrics.NumDeletes"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":149,"metricType":"sum"},{"name":"number of input batches","accumulatorId":150,"metricType":"sum"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":148,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"time to broadcast","accumulatorId":147,"metricType":"timing"},{"name":"time to build","accumulatorId":146,"metricType":"timing"},{"name":"time to collect","accumulatorId":145,"metricType":"timing"},{"name":"number of output rows","accumulatorId":144,"metricType":"sum"},{"name":"data size","accumulatorId":143,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":177,"metricType":"sum"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":176,"metricType":"timing"}]}],"metadata":{},"metrics":[]}} +{"Event":"SparkListenerJobStart","Job ID":5,"Submission Time":1714515297415,"Stage Infos":[{"Stage ID":7,"Stage Attempt ID":0,"Stage Name":"processLine at CliDriver.java:336","Number of Tasks":1,"RDD Info":[{"RDD ID":15,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"36\",\"name\":\"mapPartitionsInternal\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[14],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"35\",\"name\":\"BatchScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"DataSourceRDD","Scope":"{\"id\":\"35\",\"name\":\"BatchScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"27\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[7],"Properties":{"spark.sql.warehouse.dir":"file:/home/holden/repos/sparklens/spark-warehouse","spark.executor.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.driver.host":"10.69.200.180","spark.eventLog.enabled":"true","spark.driver.port":"37317","__fetch_continuous_blocks_in_batch_enabled":"true","spark.sql.catalog.local.type":"hadoop","spark.jars":"","spark.sql.catalog.spark_catalog":"org.apache.iceberg.spark.SparkSessionCatalog","spark.sql.datetime.java8API.enabled":"true","spark.sql.hive.version":"2.3.9","spark.app.name":"fresh","spark.rdd.scope":"{\"id\":\"37\",\"name\":\"collect\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.submitTime":"1714515291527","spark.job.description":"\nSELECT * FROM local.developers INNER JOIN local.projects ON local.projects.creator = local.developers.username","spark.app.startTime":"1714515291883","spark.executor.id":"driver","spark.sql.catalog.local.warehouse":"/home/holden/repos/sparklens/warehouse","spark.driver.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.sql.catalog.local":"org.apache.iceberg.spark.SparkCatalog","spark.submit.deployMode":"client","spark.master":"local[5]","spark.sql.execution.id":"8","spark.sql.catalogImplementation":"hive","spark.sql.catalog.spark_catalog.type":"hive","spark.app.id":"local-1714515292260"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":7,"Stage Attempt ID":0,"Stage Name":"processLine at CliDriver.java:336","Number of Tasks":1,"RDD Info":[{"RDD ID":15,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"36\",\"name\":\"mapPartitionsInternal\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[14],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"35\",\"name\":\"BatchScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"DataSourceRDD","Scope":"{\"id\":\"35\",\"name\":\"BatchScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"27\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1714515297415,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/holden/repos/sparklens/spark-warehouse","spark.executor.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.driver.host":"10.69.200.180","spark.eventLog.enabled":"true","spark.driver.port":"37317","__fetch_continuous_blocks_in_batch_enabled":"true","spark.sql.catalog.local.type":"hadoop","spark.jars":"","spark.sql.catalog.spark_catalog":"org.apache.iceberg.spark.SparkSessionCatalog","spark.sql.datetime.java8API.enabled":"true","spark.sql.hive.version":"2.3.9","spark.app.name":"fresh","spark.rdd.scope":"{\"id\":\"37\",\"name\":\"collect\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.submitTime":"1714515291527","spark.job.description":"\nSELECT * FROM local.developers INNER JOIN local.projects ON local.projects.creator = local.developers.username","spark.app.startTime":"1714515291883","spark.executor.id":"driver","spark.sql.catalog.local.warehouse":"/home/holden/repos/sparklens/warehouse","spark.driver.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.sql.catalog.local":"org.apache.iceberg.spark.SparkCatalog","spark.submit.deployMode":"client","spark.master":"local[5]","spark.sql.execution.id":"8","spark.sql.catalogImplementation":"hive","spark.sql.catalog.spark_catalog.type":"hive","spark.app.id":"local-1714515292260"}} +{"Event":"SparkListenerTaskStart","Stage ID":7,"Stage Attempt ID":0,"Task Info":{"Task ID":403,"Index":0,"Attempt":0,"Partition ID":0,"Launch Time":1714515297422,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":7,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":403,"Index":0,"Attempt":0,"Partition ID":0,"Launch Time":1714515297422,"Executor ID":"driver","Host":"10.69.200.180","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515297480,"Failed":false,"Killed":false,"Accumulables":[{"ID":131,"Name":"number of output rows","Update":"30","Value":"30","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":132,"Name":"number of file splits read","Update":"30","Value":"30","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":176,"Name":"duration","Update":"54","Value":"54","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":177,"Name":"number of output rows","Update":"900","Value":"900","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":178,"Name":"number of output rows","Update":"30","Value":"30","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":179,"Name":"number of input batches","Update":"30","Value":"30","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":180,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":181,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1322651,"Value":1322651,"Internal":true,"Count Failed Values":true},{"ID":182,"Name":"internal.metrics.executorRunTime","Update":55,"Value":55,"Internal":true,"Count Failed Values":true},{"ID":183,"Name":"internal.metrics.executorCpuTime","Update":53631711,"Value":53631711,"Internal":true,"Count Failed Values":true},{"ID":184,"Name":"internal.metrics.resultSize","Update":2629,"Value":2629,"Internal":true,"Count Failed Values":true},{"ID":189,"Name":"internal.metrics.peakExecutionMemory","Update":4195312,"Value":4195312,"Internal":true,"Count Failed Values":true},{"ID":201,"Name":"internal.metrics.input.bytesRead","Update":69480,"Value":69480,"Internal":true,"Count Failed Values":true},{"ID":202,"Name":"internal.metrics.input.recordsRead","Update":30,"Value":30,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1322651,"Executor Run Time":55,"Executor CPU Time":53631711,"Peak Execution Memory":4195312,"Result Size":2629,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":69480,"Records Read":30},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":7,"Stage Attempt ID":0,"Stage Name":"processLine at CliDriver.java:336","Number of Tasks":1,"RDD Info":[{"RDD ID":15,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"36\",\"name\":\"mapPartitionsInternal\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[14],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"35\",\"name\":\"BatchScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"DataSourceRDD","Scope":"{\"id\":\"35\",\"name\":\"BatchScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"27\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1714515297415,"Completion Time":1714515297481,"Accumulables":[{"ID":131,"Name":"number of output rows","Value":"30","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":132,"Name":"number of file splits read","Value":"30","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":176,"Name":"duration","Value":"54","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":177,"Name":"number of output rows","Value":"900","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":178,"Name":"number of output rows","Value":"30","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":179,"Name":"number of input batches","Value":"30","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":180,"Name":"internal.metrics.executorDeserializeTime","Value":1,"Internal":true,"Count Failed Values":true},{"ID":181,"Name":"internal.metrics.executorDeserializeCpuTime","Value":1322651,"Internal":true,"Count Failed Values":true},{"ID":182,"Name":"internal.metrics.executorRunTime","Value":55,"Internal":true,"Count Failed Values":true},{"ID":183,"Name":"internal.metrics.executorCpuTime","Value":53631711,"Internal":true,"Count Failed Values":true},{"ID":184,"Name":"internal.metrics.resultSize","Value":2629,"Internal":true,"Count Failed Values":true},{"ID":189,"Name":"internal.metrics.peakExecutionMemory","Value":4195312,"Internal":true,"Count Failed Values":true},{"ID":201,"Name":"internal.metrics.input.bytesRead","Value":69480,"Internal":true,"Count Failed Values":true},{"ID":202,"Name":"internal.metrics.input.recordsRead","Value":30,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":5,"Completion Time":1714515297481,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":8,"time":1714515297492} +{"Event":"SparkListenerApplicationEnd","Timestamp":1714515297508} diff --git a/src/test/event-history-test-files/local-fresh-dynamic b/src/test/event-history-test-files/local-fresh-dynamic new file mode 100644 index 0000000..ea6c8a1 --- /dev/null +++ b/src/test/event-history-test-files/local-fresh-dynamic @@ -0,0 +1,867 @@ +{"Event":"SparkListenerLogStart","Spark Version":"3.3.4"} +{"Event":"SparkListenerResourceProfileAdded","Resource Profile Id":0,"Executor Resource Requests":{"cores":{"Resource Name":"cores","Amount":1,"Discovery Script":"","Vendor":""},"memory":{"Resource Name":"memory","Amount":1024,"Discovery Script":"","Vendor":""},"offHeap":{"Resource Name":"offHeap","Amount":0,"Discovery Script":"","Vendor":""}},"Task Resource Requests":{"cpus":{"Resource Name":"cpus","Amount":1.0}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"10.69.200.180","Port":39401},"Maximum Memory":455501414,"Timestamp":1714515305321,"Maximum Onheap Memory":455501414,"Maximum Offheap Memory":0} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/lib/jvm/java-17-openjdk-amd64","Java Version":"17.0.10 (Private Build)","Scala Version":"version 2.12.15"},"Spark Properties":{"spark.dynamicAllocation.shuffleTracking.enabled":"true","spark.executor.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.driver.host":"10.69.200.180","spark.eventLog.enabled":"true","spark.driver.port":"37323","spark.sql.catalog.local.type":"hadoop","spark.jars":"","spark.sql.catalog.spark_catalog":"org.apache.iceberg.spark.SparkSessionCatalog","spark.kubernetes.authenticate.submission.caCertFile":"/var/snap/microk8s/current/certs/ca.crt","spark.sql.datetime.java8API.enabled":"true","spark.sql.hive.version":"2.3.9","spark.app.name":"fresh-kube","spark.scheduler.mode":"FIFO","spark.submit.pyFiles":"","spark.app.submitTime":"1714515303940","spark.kubernetes.container.image":"localhost:32000/local-spark/spark:magic","spark.app.startTime":"1714515304235","spark.executor.id":"driver","spark.sql.catalog.local.warehouse":"/tmp/warehouse","spark.driver.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.sql.catalog.local":"org.apache.iceberg.spark.SparkCatalog","spark.submit.deployMode":"client","spark.master":"k8s://https://127.0.0.1:16443","spark.kubernetes.executor.podNamePrefix":"fresh-kube-bb0c5b8f311360be","spark.dynamicAllocation.enabled":"true","spark.sql.catalogImplementation":"hive","spark.kubernetes.authenticate.caCertFile":"/var/snap/microk8s/current/certs/ca.crt","spark.sql.catalog.spark_catalog.type":"hive","spark.app.id":"spark-a8d39974ec7b451a92b303fa953434e4"},"Hadoop Properties":{"hadoop.service.shutdown.timeout":"30s","yarn.resourcemanager.amlauncher.thread-count":"50","yarn.sharedcache.enabled":"false","fs.s3a.connection.maximum":"96","yarn.nodemanager.numa-awareness.numactl.cmd":"/usr/bin/numactl","fs.viewfs.overload.scheme.target.o3fs.impl":"org.apache.hadoop.fs.ozone.OzoneFileSystem","fs.s3a.impl":"org.apache.hadoop.fs.s3a.S3AFileSystem","yarn.app.mapreduce.am.scheduler.heartbeat.interval-ms":"1000","yarn.timeline-service.timeline-client.number-of-async-entities-to-merge":"10","hadoop.security.kms.client.timeout":"60","hadoop.http.authentication.kerberos.principal":"HTTP/_HOST@LOCALHOST","mapreduce.jobhistory.loadedjob.tasks.max":"-1","yarn.resourcemanager.application-tag-based-placement.enable":"false","mapreduce.framework.name":"local","yarn.sharedcache.uploader.server.thread-count":"50","yarn.nodemanager.log-aggregation.roll-monitoring-interval-seconds.min":"3600","yarn.nodemanager.linux-container-executor.nonsecure-mode.user-pattern":"^[_.A-Za-z0-9][-@_.A-Za-z0-9]{0,255}?[$]?$","tfile.fs.output.buffer.size":"262144","yarn.app.mapreduce.am.job.task.listener.thread-count":"30","yarn.nodemanager.node-attributes.resync-interval-ms":"120000","yarn.nodemanager.container-log-monitor.interval-ms":"60000","hadoop.security.groups.cache.background.reload.threads":"3","yarn.resourcemanager.webapp.cross-origin.enabled":"false","fs.AbstractFileSystem.ftp.impl":"org.apache.hadoop.fs.ftp.FtpFs","fs.viewfs.overload.scheme.target.gs.impl":"com.google.cloud.hadoop.fs.gcs.GoogleHadoopFS","hadoop.registry.secure":"false","hadoop.shell.safely.delete.limit.num.files":"100","mapreduce.job.acl-view-job":" ","fs.s3a.s3guard.ddb.background.sleep":"25ms","fs.s3a.retry.limit":"7","mapreduce.jobhistory.loadedjobs.cache.size":"5","fs.s3a.s3guard.ddb.table.create":"false","fs.viewfs.overload.scheme.target.s3a.impl":"org.apache.hadoop.fs.s3a.S3AFileSystem","yarn.nodemanager.amrmproxy.enabled":"false","yarn.timeline-service.entity-group-fs-store.with-user-dir":"false","mapreduce.shuffle.pathcache.expire-after-access-minutes":"5","mapreduce.input.fileinputformat.split.minsize":"0","yarn.resourcemanager.container.liveness-monitor.interval-ms":"600000","yarn.resourcemanager.client.thread-count":"50","io.seqfile.compress.blocksize":"1000000","yarn.nodemanager.runtime.linux.docker.allowed-container-runtimes":"runc","fs.viewfs.overload.scheme.target.http.impl":"org.apache.hadoop.fs.http.HttpFileSystem","yarn.resourcemanager.nodemanagers.heartbeat-interval-slowdown-factor":"1.0","yarn.sharedcache.checksum.algo.impl":"org.apache.hadoop.yarn.sharedcache.ChecksumSHA256Impl","yarn.nodemanager.amrmproxy.interceptor-class.pipeline":"org.apache.hadoop.yarn.server.nodemanager.amrmproxy.DefaultRequestInterceptor","yarn.timeline-service.entity-group-fs-store.leveldb-cache-read-cache-size":"10485760","mapreduce.reduce.shuffle.fetch.retry.interval-ms":"1000","mapreduce.task.profile.maps":"0-2","yarn.scheduler.include-port-in-node-name":"false","yarn.nodemanager.admin-env":"MALLOC_ARENA_MAX=$MALLOC_ARENA_MAX","yarn.resourcemanager.node-removal-untracked.timeout-ms":"60000","mapreduce.am.max-attempts":"2","hadoop.security.kms.client.failover.sleep.base.millis":"100","mapreduce.jobhistory.webapp.https.address":"0.0.0.0:19890","yarn.node-labels.fs-store.impl.class":"org.apache.hadoop.yarn.nodelabels.FileSystemNodeLabelsStore","yarn.nodemanager.collector-service.address":"${yarn.nodemanager.hostname}:8048","fs.trash.checkpoint.interval":"0","mapreduce.job.map.output.collector.class":"org.apache.hadoop.mapred.MapTask$MapOutputBuffer","yarn.resourcemanager.node-ip-cache.expiry-interval-secs":"-1","hadoop.http.authentication.signature.secret.file":"*********(redacted)","hadoop.jetty.logs.serve.aliases":"true","yarn.resourcemanager.placement-constraints.handler":"disabled","yarn.timeline-service.handler-thread-count":"10","yarn.resourcemanager.max-completed-applications":"1000","yarn.nodemanager.aux-services.manifest.enabled":"false","yarn.resourcemanager.system-metrics-publisher.enabled":"false","yarn.resourcemanager.placement-constraints.algorithm.class":"org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.algorithm.DefaultPlacementAlgorithm","yarn.sharedcache.webapp.address":"0.0.0.0:8788","fs.s3a.select.input.csv.quote.escape.character":"\\\\","yarn.resourcemanager.delegation.token.renew-interval":"*********(redacted)","yarn.sharedcache.nm.uploader.replication.factor":"10","hadoop.security.groups.negative-cache.secs":"30","yarn.app.mapreduce.task.container.log.backups":"0","mapreduce.reduce.skip.proc-count.auto-incr":"true","fs.viewfs.overload.scheme.target.swift.impl":"org.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem","hadoop.security.group.mapping.ldap.posix.attr.gid.name":"gidNumber","ipc.client.fallback-to-simple-auth-allowed":"false","yarn.nodemanager.resource.memory.enforced":"true","yarn.resourcemanager.system-metrics-publisher.timeline-server-v1.enable-batch":"false","yarn.client.failover-proxy-provider":"org.apache.hadoop.yarn.client.ConfiguredRMFailoverProxyProvider","yarn.timeline-service.http-authentication.simple.anonymous.allowed":"true","ha.health-monitor.check-interval.ms":"1000","yarn.nodemanager.runtime.linux.runc.host-pid-namespace.allowed":"false","hadoop.metrics.jvm.use-thread-mxbean":"false","ipc.[port_number].faircallqueue.multiplexer.weights":"8,4,2,1","yarn.acl.reservation-enable":"false","yarn.resourcemanager.store.class":"org.apache.hadoop.yarn.server.resourcemanager.recovery.FileSystemRMStateStore","yarn.app.mapreduce.am.hard-kill-timeout-ms":"10000","fs.s3a.etag.checksum.enabled":"false","yarn.nodemanager.container-metrics.enable":"true","ha.health-monitor.rpc.connect.max.retries":"1","yarn.timeline-service.client.fd-clean-interval-secs":"60","yarn.resourcemanager.nodemanagers.heartbeat-interval-scaling-enable":"false","yarn.resourcemanager.nodemanagers.heartbeat-interval-ms":"1000","hadoop.common.configuration.version":"3.0.0","fs.s3a.s3guard.ddb.table.capacity.read":"0","yarn.nodemanager.remote-app-log-dir-suffix":"logs","yarn.nodemanager.container-log-monitor.dir-size-limit-bytes":"1000000000","yarn.nodemanager.windows-container.cpu-limit.enabled":"false","yarn.nodemanager.runtime.linux.docker.privileged-containers.allowed":"false","file.blocksize":"67108864","hadoop.http.idle_timeout.ms":"60000","hadoop.registry.zk.retry.ceiling.ms":"60000","yarn.scheduler.configuration.leveldb-store.path":"${hadoop.tmp.dir}/yarn/system/confstore","yarn.sharedcache.store.in-memory.initial-delay-mins":"10","mapreduce.jobhistory.principal":"jhs/_HOST@REALM.TLD","mapreduce.map.skip.proc-count.auto-incr":"true","fs.s3a.committer.name":"file","mapreduce.task.profile.reduces":"0-2","hadoop.zk.num-retries":"1000","yarn.webapp.xfs-filter.enabled":"true","fs.viewfs.overload.scheme.target.hdfs.impl":"org.apache.hadoop.hdfs.DistributedFileSystem","seq.io.sort.mb":"100","yarn.scheduler.configuration.max.version":"100","yarn.timeline-service.webapp.https.address":"${yarn.timeline-service.hostname}:8190","yarn.resourcemanager.scheduler.address":"${yarn.resourcemanager.hostname}:8030","yarn.node-labels.enabled":"false","yarn.resourcemanager.webapp.ui-actions.enabled":"true","mapreduce.task.timeout":"600000","yarn.sharedcache.client-server.thread-count":"50","hadoop.security.groups.shell.command.timeout":"0s","hadoop.security.crypto.cipher.suite":"AES/CTR/NoPadding","yarn.nodemanager.elastic-memory-control.oom-handler":"org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.DefaultOOMHandler","yarn.resourcemanager.connect.max-wait.ms":"900000","fs.defaultFS":"file:///","yarn.minicluster.use-rpc":"false","ipc.[port_number].decay-scheduler.decay-factor":"0.5","fs.har.impl.disable.cache":"true","yarn.webapp.ui2.enable":"false","io.compression.codec.bzip2.library":"system-native","yarn.webapp.filter-invalid-xml-chars":"false","yarn.nodemanager.runtime.linux.runc.layer-mounts-interval-secs":"600","fs.s3a.select.input.csv.record.delimiter":"\\n","fs.s3a.change.detection.source":"etag","ipc.[port_number].backoff.enable":"false","yarn.nodemanager.distributed-scheduling.enabled":"false","mapreduce.shuffle.connection-keep-alive.timeout":"5","yarn.resourcemanager.webapp.https.address":"${yarn.resourcemanager.hostname}:8090","yarn.webapp.enable-rest-app-submissions":"true","mapreduce.jobhistory.address":"0.0.0.0:10020","yarn.resourcemanager.nm-tokens.master-key-rolling-interval-secs":"*********(redacted)","yarn.is.minicluster":"false","yarn.nodemanager.address":"${yarn.nodemanager.hostname}:0","fs.abfss.impl":"org.apache.hadoop.fs.azurebfs.SecureAzureBlobFileSystem","fs.AbstractFileSystem.s3a.impl":"org.apache.hadoop.fs.s3a.S3A","mapreduce.task.combine.progress.records":"10000","yarn.resourcemanager.epoch.range":"0","yarn.resourcemanager.am.max-attempts":"2","yarn.nodemanager.runtime.linux.runc.image-toplevel-dir":"/runc-root","yarn.nodemanager.linux-container-executor.cgroups.hierarchy":"/hadoop-yarn","fs.AbstractFileSystem.wasbs.impl":"org.apache.hadoop.fs.azure.Wasbs","yarn.timeline-service.entity-group-fs-store.cache-store-class":"org.apache.hadoop.yarn.server.timeline.MemoryTimelineStore","yarn.nodemanager.runtime.linux.runc.allowed-container-networks":"host,none,bridge","fs.ftp.transfer.mode":"BLOCK_TRANSFER_MODE","ipc.server.log.slow.rpc":"false","ipc.server.reuseaddr":"true","fs.ftp.timeout":"0","yarn.resourcemanager.node-labels.provider.fetch-interval-ms":"1800000","yarn.router.webapp.https.address":"0.0.0.0:8091","yarn.nodemanager.webapp.cross-origin.enabled":"false","fs.wasb.impl":"org.apache.hadoop.fs.azure.NativeAzureFileSystem","yarn.resourcemanager.auto-update.containers":"false","yarn.app.mapreduce.am.job.committer.cancel-timeout":"60000","yarn.scheduler.configuration.zk-store.parent-path":"/confstore","yarn.nodemanager.default-container-executor.log-dirs.permissions":"710","yarn.app.attempt.diagnostics.limit.kc":"64","fs.viewfs.overload.scheme.target.swebhdfs.impl":"org.apache.hadoop.hdfs.web.SWebHdfsFileSystem","yarn.client.failover-no-ha-proxy-provider":"org.apache.hadoop.yarn.client.DefaultNoHARMFailoverProxyProvider","fs.s3a.change.detection.mode":"server","ftp.bytes-per-checksum":"512","yarn.nodemanager.resource.memory-mb":"-1","fs.AbstractFileSystem.abfs.impl":"org.apache.hadoop.fs.azurebfs.Abfs","yarn.timeline-service.writer.flush-interval-seconds":"60","fs.s3a.fast.upload.active.blocks":"4","yarn.resourcemanager.submission-preprocessor.enabled":"false","hadoop.security.credential.clear-text-fallback":"true","yarn.nodemanager.collector-service.thread-count":"5","ipc.[port_number].scheduler.impl":"org.apache.hadoop.ipc.DefaultRpcScheduler","fs.azure.secure.mode":"false","mapreduce.jobhistory.joblist.cache.size":"20000","fs.ftp.host":"0.0.0.0","yarn.timeline-service.writer.async.queue.capacity":"100","yarn.resourcemanager.fs.state-store.num-retries":"0","yarn.resourcemanager.nodemanager-connect-retries":"10","yarn.nodemanager.log-aggregation.num-log-files-per-app":"30","hadoop.security.kms.client.encrypted.key.cache.low-watermark":"0.3f","fs.s3a.committer.magic.enabled":"true","yarn.timeline-service.client.max-retries":"30","dfs.ha.fencing.ssh.connect-timeout":"30000","yarn.log-aggregation-enable":"false","yarn.system-metrics-publisher.enabled":"false","mapreduce.reduce.markreset.buffer.percent":"0.0","fs.AbstractFileSystem.viewfs.impl":"org.apache.hadoop.fs.viewfs.ViewFs","yarn.resourcemanager.nodemanagers.heartbeat-interval-speedup-factor":"1.0","mapreduce.task.io.sort.factor":"10","yarn.nodemanager.amrmproxy.client.thread-count":"25","ha.failover-controller.new-active.rpc-timeout.ms":"60000","yarn.nodemanager.container-localizer.java.opts":"-Xmx256m","mapreduce.jobhistory.datestring.cache.size":"200000","mapreduce.job.acl-modify-job":" ","yarn.nodemanager.windows-container.memory-limit.enabled":"false","yarn.timeline-service.webapp.address":"${yarn.timeline-service.hostname}:8188","yarn.app.mapreduce.am.job.committer.commit-window":"10000","yarn.nodemanager.container-manager.thread-count":"20","yarn.minicluster.fixed.ports":"false","hadoop.tags.system":"YARN,HDFS,NAMENODE,DATANODE,REQUIRED,SECURITY,KERBEROS,PERFORMANCE,CLIENT\n ,SERVER,DEBUG,DEPRECATED,COMMON,OPTIONAL","yarn.cluster.max-application-priority":"0","yarn.timeline-service.ttl-enable":"true","mapreduce.jobhistory.recovery.store.fs.uri":"${hadoop.tmp.dir}/mapred/history/recoverystore","hadoop.caller.context.signature.max.size":"40","ipc.[port_number].decay-scheduler.backoff.responsetime.enable":"false","yarn.client.load.resource-types.from-server":"false","ha.zookeeper.session-timeout.ms":"10000","ipc.[port_number].decay-scheduler.metrics.top.user.count":"10","tfile.io.chunk.size":"1048576","fs.s3a.s3guard.ddb.table.capacity.write":"0","yarn.dispatcher.print-events-info.threshold":"5000","mapreduce.job.speculative.slowtaskthreshold":"1.0","io.serializations":"org.apache.hadoop.io.serializer.WritableSerialization, org.apache.hadoop.io.serializer.avro.AvroSpecificSerialization, org.apache.hadoop.io.serializer.avro.AvroReflectSerialization","hadoop.security.kms.client.failover.sleep.max.millis":"2000","hadoop.security.group.mapping.ldap.directory.search.timeout":"10000","yarn.scheduler.configuration.store.max-logs":"1000","yarn.nodemanager.node-attributes.provider.fetch-interval-ms":"600000","fs.swift.impl":"org.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem","yarn.nodemanager.local-cache.max-files-per-directory":"8192","hadoop.http.cross-origin.enabled":"false","hadoop.zk.acl":"world:anyone:rwcda","yarn.nodemanager.runtime.linux.runc.image-tag-to-manifest-plugin.num-manifests-to-cache":"10","mapreduce.map.sort.spill.percent":"0.80","yarn.timeline-service.entity-group-fs-store.scan-interval-seconds":"60","yarn.node-attribute.fs-store.impl.class":"org.apache.hadoop.yarn.server.resourcemanager.nodelabels.FileSystemNodeAttributeStore","fs.s3a.retry.interval":"500ms","yarn.timeline-service.client.best-effort":"false","yarn.resourcemanager.webapp.delegation-token-auth-filter.enabled":"*********(redacted)","hadoop.security.group.mapping.ldap.posix.attr.uid.name":"uidNumber","fs.AbstractFileSystem.swebhdfs.impl":"org.apache.hadoop.fs.SWebHdfs","yarn.nodemanager.elastic-memory-control.timeout-sec":"5","fs.s3a.select.enabled":"true","mapreduce.ifile.readahead":"true","yarn.timeline-service.leveldb-timeline-store.ttl-interval-ms":"300000","yarn.timeline-service.reader.webapp.address":"${yarn.timeline-service.webapp.address}","yarn.resourcemanager.placement-constraints.algorithm.pool-size":"1","yarn.timeline-service.hbase.coprocessor.jar.hdfs.location":"/hbase/coprocessor/hadoop-yarn-server-timelineservice.jar","hadoop.security.kms.client.encrypted.key.cache.num.refill.threads":"2","yarn.resourcemanager.scheduler.class":"org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler","yarn.app.mapreduce.am.command-opts":"-Xmx1024m","fs.s3a.metadatastore.fail.on.write.error":"true","hadoop.http.sni.host.check.enabled":"false","mapreduce.cluster.local.dir":"${hadoop.tmp.dir}/mapred/local","io.mapfile.bloom.error.rate":"0.005","fs.client.resolve.topology.enabled":"false","yarn.nodemanager.runtime.linux.allowed-runtimes":"default","yarn.sharedcache.store.class":"org.apache.hadoop.yarn.server.sharedcachemanager.store.InMemorySCMStore","ha.failover-controller.graceful-fence.rpc-timeout.ms":"5000","ftp.replication":"3","fs.getspaceused.jitterMillis":"60000","hadoop.security.uid.cache.secs":"14400","mapreduce.job.maxtaskfailures.per.tracker":"3","fs.s3a.metadatastore.impl":"org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore","io.skip.checksum.errors":"false","yarn.app.mapreduce.client-am.ipc.max-retries-on-timeouts":"3","yarn.timeline-service.webapp.xfs-filter.xframe-options":"SAMEORIGIN","fs.s3a.connection.timeout":"200000","yarn.app.mapreduce.am.webapp.https.enabled":"false","mapreduce.job.max.split.locations":"15","yarn.resourcemanager.nm-container-queuing.max-queue-length":"15","yarn.resourcemanager.delegation-token.always-cancel":"*********(redacted)","hadoop.registry.zk.session.timeout.ms":"60000","yarn.federation.cache-ttl.secs":"300","mapreduce.jvm.system-properties-to-log":"os.name,os.version,java.home,java.runtime.version,java.vendor,java.version,java.vm.name,java.class.path,java.io.tmpdir,user.dir,user.name","yarn.resourcemanager.opportunistic-container-allocation.nodes-used":"10","yarn.timeline-service.entity-group-fs-store.active-dir":"/tmp/entity-file-history/active","mapreduce.shuffle.transfer.buffer.size":"131072","yarn.timeline-service.client.retry-interval-ms":"1000","yarn.timeline-service.flowname.max-size":"0","yarn.http.policy":"HTTP_ONLY","fs.s3a.socket.send.buffer":"8192","fs.AbstractFileSystem.abfss.impl":"org.apache.hadoop.fs.azurebfs.Abfss","yarn.sharedcache.uploader.server.address":"0.0.0.0:8046","yarn.resourcemanager.delegation-token.max-conf-size-bytes":"*********(redacted)","hadoop.http.authentication.token.validity":"*********(redacted)","mapreduce.shuffle.max.connections":"0","yarn.minicluster.yarn.nodemanager.resource.memory-mb":"4096","mapreduce.job.emit-timeline-data":"false","yarn.nodemanager.resource.system-reserved-memory-mb":"-1","hadoop.kerberos.min.seconds.before.relogin":"60","mapreduce.jobhistory.move.thread-count":"3","yarn.resourcemanager.admin.client.thread-count":"1","yarn.dispatcher.drain-events.timeout":"300000","ipc.[port_number].decay-scheduler.backoff.responsetime.thresholds":"10s,20s,30s,40s","fs.s3a.buffer.dir":"${hadoop.tmp.dir}/s3a","hadoop.ssl.enabled.protocols":"TLSv1.2","mapreduce.jobhistory.admin.address":"0.0.0.0:10033","yarn.log-aggregation-status.time-out.ms":"600000","fs.s3a.accesspoint.required":"false","mapreduce.shuffle.port":"13562","yarn.resourcemanager.max-log-aggregation-diagnostics-in-memory":"10","yarn.nodemanager.health-checker.interval-ms":"600000","yarn.router.clientrm.interceptor-class.pipeline":"org.apache.hadoop.yarn.server.router.clientrm.DefaultClientRequestInterceptor","yarn.resourcemanager.zk-appid-node.split-index":"0","ftp.blocksize":"67108864","yarn.nodemanager.runtime.linux.sandbox-mode.local-dirs.permissions":"read","yarn.router.rmadmin.interceptor-class.pipeline":"org.apache.hadoop.yarn.server.router.rmadmin.DefaultRMAdminRequestInterceptor","yarn.nodemanager.log-container-debug-info.enabled":"true","yarn.resourcemanager.activities-manager.app-activities.max-queue-length":"100","yarn.resourcemanager.application-https.policy":"NONE","yarn.client.max-cached-nodemanagers-proxies":"0","yarn.nodemanager.linux-container-executor.cgroups.delete-delay-ms":"20","yarn.nodemanager.delete.debug-delay-sec":"0","yarn.nodemanager.pmem-check-enabled":"true","yarn.nodemanager.disk-health-checker.max-disk-utilization-per-disk-percentage":"90.0","mapreduce.app-submission.cross-platform":"false","yarn.resourcemanager.work-preserving-recovery.scheduling-wait-ms":"10000","yarn.nodemanager.container-retry-minimum-interval-ms":"1000","hadoop.security.groups.cache.secs":"300","yarn.federation.enabled":"false","yarn.workflow-id.tag-prefix":"workflowid:","fs.azure.local.sas.key.mode":"false","ipc.maximum.data.length":"134217728","fs.s3a.endpoint":"s3.amazonaws.com","mapreduce.shuffle.max.threads":"0","yarn.router.pipeline.cache-max-size":"25","yarn.resourcemanager.nm-container-queuing.load-comparator":"QUEUE_LENGTH","yarn.resourcemanager.resource-tracker.nm.ip-hostname-check":"false","hadoop.security.authorization":"false","mapreduce.job.complete.cancel.delegation.tokens":"*********(redacted)","fs.s3a.paging.maximum":"5000","nfs.exports.allowed.hosts":"* rw","yarn.nodemanager.amrmproxy.ha.enable":"false","fs.AbstractFileSystem.gs.impl":"com.google.cloud.hadoop.fs.gcs.GoogleHadoopFS","mapreduce.jobhistory.http.policy":"HTTP_ONLY","yarn.sharedcache.store.in-memory.check-period-mins":"720","hadoop.security.group.mapping.ldap.ssl":"false","fs.s3a.downgrade.syncable.exceptions":"true","yarn.client.application-client-protocol.poll-interval-ms":"200","yarn.scheduler.configuration.leveldb-store.compaction-interval-secs":"86400","yarn.timeline-service.writer.class":"org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineWriterImpl","ha.zookeeper.parent-znode":"/hadoop-ha","yarn.resourcemanager.submission-preprocessor.file-refresh-interval-ms":"60000","yarn.nodemanager.log-aggregation.policy.class":"org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.AllContainerLogAggregationPolicy","mapreduce.reduce.shuffle.merge.percent":"0.66","hadoop.security.group.mapping.ldap.search.filter.group":"(objectClass=group)","yarn.resourcemanager.placement-constraints.scheduler.pool-size":"1","yarn.resourcemanager.activities-manager.cleanup-interval-ms":"5000","yarn.nodemanager.resourcemanager.minimum.version":"NONE","mapreduce.job.speculative.speculative-cap-running-tasks":"0.1","yarn.admin.acl":"*","ipc.[port_number].identity-provider.impl":"org.apache.hadoop.ipc.UserIdentityProvider","yarn.nodemanager.recovery.supervised":"false","yarn.sharedcache.admin.thread-count":"1","yarn.resourcemanager.ha.automatic-failover.enabled":"true","yarn.nodemanager.container-log-monitor.total-size-limit-bytes":"10000000000","mapreduce.reduce.skip.maxgroups":"0","mapreduce.reduce.shuffle.connect.timeout":"180000","yarn.nodemanager.health-checker.scripts":"script","yarn.resourcemanager.address":"${yarn.resourcemanager.hostname}:8032","ipc.client.ping":"true","mapreduce.task.local-fs.write-limit.bytes":"-1","fs.adl.oauth2.access.token.provider.type":"*********(redacted)","mapreduce.shuffle.ssl.file.buffer.size":"65536","yarn.resourcemanager.ha.automatic-failover.embedded":"true","yarn.nodemanager.resource-plugins.gpu.docker-plugin":"nvidia-docker-v1","fs.s3a.s3guard.consistency.retry.interval":"2s","fs.s3a.multipart.purge":"false","yarn.scheduler.configuration.store.class":"file","yarn.resourcemanager.nm-container-queuing.queue-limit-stdev":"1.0f","mapreduce.job.end-notification.max.attempts":"5","mapreduce.output.fileoutputformat.compress.codec":"org.apache.hadoop.io.compress.DefaultCodec","yarn.nodemanager.container-monitor.procfs-tree.smaps-based-rss.enabled":"false","ipc.client.bind.wildcard.addr":"false","yarn.resourcemanager.webapp.rest-csrf.enabled":"false","ha.health-monitor.connect-retry-interval.ms":"1000","yarn.nodemanager.keytab":"/etc/krb5.keytab","mapreduce.jobhistory.keytab":"/etc/security/keytab/jhs.service.keytab","fs.s3a.threads.max":"64","yarn.nodemanager.runtime.linux.docker.image-update":"false","mapreduce.reduce.shuffle.input.buffer.percent":"0.70","fs.viewfs.overload.scheme.target.abfss.impl":"org.apache.hadoop.fs.azurebfs.SecureAzureBlobFileSystem","yarn.dispatcher.cpu-monitor.samples-per-min":"60","hadoop.security.token.service.use_ip":"*********(redacted)","yarn.nodemanager.runtime.linux.docker.allowed-container-networks":"host,none,bridge","yarn.nodemanager.node-labels.resync-interval-ms":"120000","hadoop.tmp.dir":"/tmp/hadoop-${user.name}","mapreduce.job.maps":"2","mapreduce.jobhistory.webapp.rest-csrf.custom-header":"X-XSRF-Header","mapreduce.job.end-notification.max.retry.interval":"5000","yarn.log-aggregation.retain-check-interval-seconds":"-1","yarn.resourcemanager.resource-tracker.client.thread-count":"50","yarn.nodemanager.containers-launcher.class":"org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainersLauncher","yarn.rm.system-metrics-publisher.emit-container-events":"false","yarn.timeline-service.leveldb-timeline-store.start-time-read-cache-size":"10000","yarn.resourcemanager.ha.automatic-failover.zk-base-path":"/yarn-leader-election","io.seqfile.local.dir":"${hadoop.tmp.dir}/io/local","fs.s3a.s3guard.ddb.throttle.retry.interval":"100ms","fs.AbstractFileSystem.wasb.impl":"org.apache.hadoop.fs.azure.Wasb","mapreduce.client.submit.file.replication":"10","mapreduce.jobhistory.minicluster.fixed.ports":"false","fs.s3a.multipart.threshold":"128M","yarn.resourcemanager.webapp.xfs-filter.xframe-options":"SAMEORIGIN","mapreduce.jobhistory.done-dir":"${yarn.app.mapreduce.am.staging-dir}/history/done","ipc.server.purge.interval":"15","ipc.client.idlethreshold":"4000","yarn.nodemanager.linux-container-executor.cgroups.strict-resource-usage":"false","mapreduce.reduce.input.buffer.percent":"0.0","yarn.nodemanager.runtime.linux.docker.userremapping-gid-threshold":"1","yarn.nodemanager.webapp.rest-csrf.enabled":"false","fs.ftp.host.port":"21","ipc.ping.interval":"60000","yarn.resourcemanager.history-writer.multi-threaded-dispatcher.pool-size":"10","yarn.resourcemanager.admin.address":"${yarn.resourcemanager.hostname}:8033","file.client-write-packet-size":"65536","ipc.client.kill.max":"10","mapreduce.reduce.speculative":"true","hadoop.security.key.default.bitlength":"128","mapreduce.job.reducer.unconditional-preempt.delay.sec":"300","yarn.nodemanager.disk-health-checker.interval-ms":"120000","yarn.nodemanager.log.deletion-threads-count":"4","fs.s3a.committer.abort.pending.uploads":"true","yarn.webapp.filter-entity-list-by-user":"false","yarn.resourcemanager.activities-manager.app-activities.ttl-ms":"600000","ipc.client.connection.maxidletime":"10000","mapreduce.task.io.sort.mb":"100","yarn.nodemanager.localizer.client.thread-count":"5","io.erasurecode.codec.rs.rawcoders":"rs_native,rs_java","io.erasurecode.codec.rs-legacy.rawcoders":"rs-legacy_java","yarn.sharedcache.admin.address":"0.0.0.0:8047","yarn.resourcemanager.placement-constraints.algorithm.iterator":"SERIAL","yarn.nodemanager.localizer.cache.cleanup.interval-ms":"600000","hadoop.security.crypto.codec.classes.aes.ctr.nopadding":"org.apache.hadoop.crypto.OpensslAesCtrCryptoCodec, org.apache.hadoop.crypto.JceAesCtrCryptoCodec","mapreduce.job.cache.limit.max-resources-mb":"0","fs.s3a.connection.ssl.enabled":"true","yarn.nodemanager.process-kill-wait.ms":"5000","mapreduce.job.hdfs-servers":"${fs.defaultFS}","yarn.app.mapreduce.am.webapp.https.client.auth":"false","hadoop.workaround.non.threadsafe.getpwuid":"true","fs.df.interval":"60000","ipc.[port_number].decay-scheduler.thresholds":"13,25,50","fs.s3a.multiobjectdelete.enable":"true","yarn.sharedcache.cleaner.resource-sleep-ms":"0","yarn.nodemanager.disk-health-checker.min-healthy-disks":"0.25","hadoop.shell.missing.defaultFs.warning":"false","io.file.buffer.size":"65536","fs.viewfs.overload.scheme.target.wasb.impl":"org.apache.hadoop.fs.azure.NativeAzureFileSystem","hadoop.security.group.mapping.ldap.search.attr.member":"member","hadoop.security.random.device.file.path":"/dev/urandom","hadoop.security.sensitive-config-keys":"*********(redacted)","fs.s3a.s3guard.ddb.max.retries":"9","fs.viewfs.overload.scheme.target.file.impl":"org.apache.hadoop.fs.LocalFileSystem","hadoop.rpc.socket.factory.class.default":"org.apache.hadoop.net.StandardSocketFactory","yarn.intermediate-data-encryption.enable":"false","yarn.resourcemanager.connect.retry-interval.ms":"30000","yarn.nodemanager.container.stderr.pattern":"{*stderr*,*STDERR*}","yarn.scheduler.minimum-allocation-mb":"1024","yarn.app.mapreduce.am.staging-dir":"/tmp/hadoop-yarn/staging","mapreduce.reduce.shuffle.read.timeout":"180000","hadoop.http.cross-origin.max-age":"1800","io.erasurecode.codec.xor.rawcoders":"xor_native,xor_java","fs.s3a.s3guard.consistency.retry.limit":"7","fs.s3a.connection.establish.timeout":"5000","mapreduce.job.running.map.limit":"0","yarn.minicluster.control-resource-monitoring":"false","hadoop.ssl.require.client.cert":"false","hadoop.kerberos.kinit.command":"kinit","yarn.federation.state-store.class":"org.apache.hadoop.yarn.server.federation.store.impl.MemoryFederationStateStore","mapreduce.reduce.log.level":"INFO","hadoop.security.dns.log-slow-lookups.threshold.ms":"1000","mapreduce.job.ubertask.enable":"false","adl.http.timeout":"-1","yarn.resourcemanager.placement-constraints.retry-attempts":"3","hadoop.caller.context.enabled":"false","hadoop.security.group.mapping.ldap.num.attempts":"3","yarn.nodemanager.vmem-pmem-ratio":"2.1","hadoop.rpc.protection":"authentication","ha.health-monitor.rpc-timeout.ms":"45000","yarn.nodemanager.remote-app-log-dir":"/tmp/logs","hadoop.zk.timeout-ms":"10000","fs.s3a.s3guard.cli.prune.age":"86400000","yarn.nodemanager.resource.pcores-vcores-multiplier":"1.0","yarn.nodemanager.runtime.linux.sandbox-mode":"disabled","yarn.app.mapreduce.am.containerlauncher.threadpool-initial-size":"10","fs.viewfs.overload.scheme.target.webhdfs.impl":"org.apache.hadoop.hdfs.web.WebHdfsFileSystem","fs.s3a.committer.threads":"8","hadoop.zk.retry-interval-ms":"1000","hadoop.security.crypto.buffer.size":"8192","yarn.nodemanager.node-labels.provider.fetch-interval-ms":"600000","mapreduce.jobhistory.recovery.store.leveldb.path":"${hadoop.tmp.dir}/mapred/history/recoverystore","yarn.client.failover-retries-on-socket-timeouts":"0","fs.s3a.ssl.channel.mode":"default_jsse","yarn.nodemanager.resource.memory.enabled":"false","fs.azure.authorization.caching.enable":"true","hadoop.security.instrumentation.requires.admin":"false","yarn.nodemanager.delete.thread-count":"4","mapreduce.job.finish-when-all-reducers-done":"true","hadoop.registry.jaas.context":"Client","yarn.timeline-service.leveldb-timeline-store.path":"${hadoop.tmp.dir}/yarn/timeline","io.map.index.interval":"128","yarn.resourcemanager.nm-container-queuing.max-queue-wait-time-ms":"100","fs.abfs.impl":"org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem","mapreduce.job.counters.max":"120","mapreduce.jobhistory.webapp.rest-csrf.enabled":"false","yarn.timeline-service.store-class":"org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore","mapreduce.jobhistory.move.interval-ms":"180000","fs.s3a.change.detection.version.required":"true","yarn.nodemanager.localizer.fetch.thread-count":"4","yarn.resourcemanager.scheduler.client.thread-count":"50","hadoop.ssl.hostname.verifier":"DEFAULT","yarn.timeline-service.leveldb-state-store.path":"${hadoop.tmp.dir}/yarn/timeline","mapreduce.job.classloader":"false","mapreduce.task.profile.map.params":"${mapreduce.task.profile.params}","ipc.client.connect.timeout":"20000","hadoop.security.auth_to_local.mechanism":"hadoop","yarn.timeline-service.app-collector.linger-period.ms":"60000","yarn.nm.liveness-monitor.expiry-interval-ms":"600000","yarn.resourcemanager.reservation-system.planfollower.time-step":"1000","yarn.resourcemanager.activities-manager.scheduler-activities.ttl-ms":"600000","yarn.nodemanager.runtime.linux.docker.enable-userremapping.allowed":"true","yarn.webapp.api-service.enable":"false","yarn.nodemanager.recovery.enabled":"false","mapreduce.job.end-notification.retry.interval":"1000","fs.du.interval":"600000","fs.ftp.impl":"org.apache.hadoop.fs.ftp.FTPFileSystem","yarn.nodemanager.container.stderr.tail.bytes":"4096","yarn.nodemanager.disk-health-checker.disk-free-space-threshold.enabled":"true","hadoop.security.group.mapping.ldap.read.timeout.ms":"60000","hadoop.security.groups.cache.warn.after.ms":"5000","file.bytes-per-checksum":"512","mapreduce.outputcommitter.factory.scheme.s3a":"org.apache.hadoop.fs.s3a.commit.S3ACommitterFactory","hadoop.security.groups.cache.background.reload":"false","yarn.nodemanager.container-monitor.enabled":"true","yarn.nodemanager.elastic-memory-control.enabled":"false","net.topology.script.number.args":"100","mapreduce.task.merge.progress.records":"10000","yarn.nodemanager.localizer.address":"${yarn.nodemanager.hostname}:8040","yarn.timeline-service.keytab":"/etc/krb5.keytab","mapreduce.reduce.shuffle.fetch.retry.timeout-ms":"30000","yarn.resourcemanager.rm.container-allocation.expiry-interval-ms":"600000","yarn.nodemanager.container-executor.exit-code-file.timeout-ms":"2000","mapreduce.fileoutputcommitter.algorithm.version":"1","yarn.resourcemanager.work-preserving-recovery.enabled":"true","mapreduce.map.skip.maxrecords":"0","yarn.sharedcache.root-dir":"/sharedcache","fs.s3a.retry.throttle.limit":"20","hadoop.http.authentication.type":"simple","fs.viewfs.overload.scheme.target.oss.impl":"org.apache.hadoop.fs.aliyun.oss.AliyunOSSFileSystem","mapreduce.job.cache.limit.max-resources":"0","mapreduce.task.userlog.limit.kb":"0","ipc.[port_number].weighted-cost.handler":"1","yarn.resourcemanager.scheduler.monitor.enable":"false","ipc.client.connect.max.retries":"10","hadoop.registry.zk.retry.times":"5","yarn.nodemanager.resource-monitor.interval-ms":"3000","yarn.nodemanager.resource-plugins.gpu.allowed-gpu-devices":"auto","mapreduce.job.sharedcache.mode":"disabled","yarn.nodemanager.webapp.rest-csrf.custom-header":"X-XSRF-Header","mapreduce.shuffle.listen.queue.size":"128","yarn.scheduler.configuration.mutation.acl-policy.class":"org.apache.hadoop.yarn.server.resourcemanager.scheduler.DefaultConfigurationMutationACLPolicy","mapreduce.map.cpu.vcores":"1","yarn.log-aggregation.file-formats":"TFile","yarn.timeline-service.client.fd-retain-secs":"300","fs.s3a.select.output.csv.field.delimiter":",","yarn.nodemanager.health-checker.timeout-ms":"1200000","hadoop.user.group.static.mapping.overrides":"dr.who=;","fs.azure.sas.expiry.period":"90d","fs.s3a.select.output.csv.record.delimiter":"\\n","mapreduce.jobhistory.recovery.store.class":"org.apache.hadoop.mapreduce.v2.hs.HistoryServerFileSystemStateStoreService","fs.viewfs.overload.scheme.target.https.impl":"org.apache.hadoop.fs.http.HttpsFileSystem","fs.s3a.s3guard.ddb.table.sse.enabled":"false","yarn.resourcemanager.fail-fast":"${yarn.fail-fast}","yarn.resourcemanager.proxy-user-privileges.enabled":"false","yarn.router.webapp.interceptor-class.pipeline":"org.apache.hadoop.yarn.server.router.webapp.DefaultRequestInterceptorREST","yarn.nodemanager.resource.memory.cgroups.soft-limit-percentage":"90.0","mapreduce.job.reducer.preempt.delay.sec":"0","hadoop.util.hash.type":"murmur","yarn.nodemanager.disk-validator":"basic","yarn.app.mapreduce.client.job.max-retries":"3","fs.viewfs.overload.scheme.target.ftp.impl":"org.apache.hadoop.fs.ftp.FTPFileSystem","mapreduce.reduce.shuffle.retry-delay.max.ms":"60000","hadoop.security.group.mapping.ldap.connection.timeout.ms":"60000","mapreduce.task.profile.params":"-agentlib:hprof=cpu=samples,heap=sites,force=n,thread=y,verbose=n,file=%s","yarn.app.mapreduce.shuffle.log.backups":"0","yarn.nodemanager.container-diagnostics-maximum-size":"10000","hadoop.registry.zk.retry.interval.ms":"1000","yarn.nodemanager.linux-container-executor.cgroups.delete-timeout-ms":"1000","fs.AbstractFileSystem.file.impl":"org.apache.hadoop.fs.local.LocalFs","yarn.nodemanager.log-aggregation.roll-monitoring-interval-seconds":"-1","mapreduce.jobhistory.cleaner.interval-ms":"86400000","hadoop.registry.zk.quorum":"localhost:2181","yarn.nodemanager.runtime.linux.runc.allowed-container-runtimes":"runc","mapreduce.output.fileoutputformat.compress":"false","yarn.resourcemanager.am-rm-tokens.master-key-rolling-interval-secs":"*********(redacted)","fs.s3a.assumed.role.session.duration":"30m","hadoop.security.group.mapping.ldap.conversion.rule":"none","hadoop.ssl.server.conf":"ssl-server.xml","fs.s3a.retry.throttle.interval":"100ms","seq.io.sort.factor":"100","fs.viewfs.overload.scheme.target.ofs.impl":"org.apache.hadoop.fs.ozone.RootedOzoneFileSystem","yarn.sharedcache.cleaner.initial-delay-mins":"10","mapreduce.client.completion.pollinterval":"5000","hadoop.ssl.keystores.factory.class":"org.apache.hadoop.security.ssl.FileBasedKeyStoresFactory","yarn.app.mapreduce.am.resource.cpu-vcores":"1","yarn.timeline-service.enabled":"false","yarn.nodemanager.runtime.linux.docker.capabilities":"CHOWN,DAC_OVERRIDE,FSETID,FOWNER,MKNOD,NET_RAW,SETGID,SETUID,SETFCAP,SETPCAP,NET_BIND_SERVICE,SYS_CHROOT,KILL,AUDIT_WRITE","yarn.acl.enable":"false","yarn.timeline-service.entity-group-fs-store.done-dir":"/tmp/entity-file-history/done/","hadoop.security.group.mapping.ldap.num.attempts.before.failover":"3","mapreduce.task.profile":"false","hadoop.prometheus.endpoint.enabled":"false","yarn.resourcemanager.fs.state-store.uri":"${hadoop.tmp.dir}/yarn/system/rmstore","mapreduce.jobhistory.always-scan-user-dir":"false","fs.s3a.metadatastore.metadata.ttl":"15m","yarn.nodemanager.opportunistic-containers-use-pause-for-preemption":"false","yarn.nodemanager.linux-container-executor.nonsecure-mode.local-user":"nobody","yarn.timeline-service.reader.class":"org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineReaderImpl","yarn.resourcemanager.configuration.provider-class":"org.apache.hadoop.yarn.LocalConfigurationProvider","yarn.nodemanager.runtime.linux.docker.userremapping-uid-threshold":"1","yarn.resourcemanager.configuration.file-system-based-store":"/yarn/conf","mapreduce.job.cache.limit.max-single-resource-mb":"0","yarn.nodemanager.runtime.linux.docker.stop.grace-period":"10","yarn.resourcemanager.resource-profiles.source-file":"resource-profiles.json","mapreduce.job.dfs.storage.capacity.kill-limit-exceed":"false","yarn.nodemanager.resource.percentage-physical-cpu-limit":"100","mapreduce.jobhistory.client.thread-count":"10","tfile.fs.input.buffer.size":"262144","mapreduce.client.progressmonitor.pollinterval":"1000","yarn.nodemanager.log-dirs":"${yarn.log.dir}/userlogs","yarn.resourcemanager.opportunistic.max.container-allocation.per.am.heartbeat":"-1","fs.automatic.close":"true","yarn.resourcemanager.delegation-token-renewer.thread-retry-interval":"*********(redacted)","fs.s3a.select.input.csv.quote.character":"\"","yarn.nodemanager.hostname":"0.0.0.0","ipc.[port_number].cost-provider.impl":"org.apache.hadoop.ipc.DefaultCostProvider","yarn.nodemanager.runtime.linux.runc.manifest-to-resources-plugin":"org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.runc.HdfsManifestToResourcesPlugin","yarn.nodemanager.remote-app-log-dir-include-older":"true","yarn.nodemanager.resource.memory.cgroups.swappiness":"0","ftp.stream-buffer-size":"4096","yarn.fail-fast":"false","yarn.nodemanager.runtime.linux.runc.layer-mounts-to-keep":"100","yarn.timeline-service.app-aggregation-interval-secs":"15","hadoop.security.group.mapping.ldap.search.filter.user":"(&(objectClass=user)(sAMAccountName={0}))","ipc.[port_number].weighted-cost.lockshared":"10","yarn.nodemanager.container-localizer.log.level":"INFO","yarn.timeline-service.address":"${yarn.timeline-service.hostname}:10200","mapreduce.job.ubertask.maxmaps":"9","fs.s3a.threads.keepalivetime":"60","mapreduce.jobhistory.webapp.rest-csrf.methods-to-ignore":"GET,OPTIONS,HEAD","mapreduce.task.files.preserve.failedtasks":"false","yarn.app.mapreduce.client.job.retry-interval":"2000","ha.failover-controller.graceful-fence.connection.retries":"1","fs.s3a.select.output.csv.quote.escape.character":"\\\\","yarn.resourcemanager.delegation.token.max-lifetime":"*********(redacted)","hadoop.kerberos.keytab.login.autorenewal.enabled":"false","yarn.timeline-service.client.drain-entities.timeout.ms":"2000","yarn.nodemanager.resource-plugins.fpga.vendor-plugin.class":"org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.fpga.IntelFpgaOpenclPlugin","yarn.resourcemanager.nodemanagers.heartbeat-interval-min-ms":"1000","yarn.timeline-service.entity-group-fs-store.summary-store":"org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore","mapreduce.reduce.cpu.vcores":"1","mapreduce.job.encrypted-intermediate-data.buffer.kb":"128","fs.client.resolve.remote.symlinks":"true","yarn.nodemanager.webapp.https.address":"0.0.0.0:8044","hadoop.http.cross-origin.allowed-origins":"*","mapreduce.job.encrypted-intermediate-data":"false","yarn.nodemanager.disk-health-checker.disk-utilization-threshold.enabled":"true","fs.s3a.executor.capacity":"16","yarn.timeline-service.entity-group-fs-store.retain-seconds":"604800","yarn.resourcemanager.metrics.runtime.buckets":"60,300,1440","yarn.timeline-service.generic-application-history.max-applications":"10000","yarn.nodemanager.local-dirs":"${hadoop.tmp.dir}/nm-local-dir","mapreduce.shuffle.connection-keep-alive.enable":"false","yarn.node-labels.configuration-type":"centralized","fs.s3a.path.style.access":"false","yarn.nodemanager.aux-services.mapreduce_shuffle.class":"org.apache.hadoop.mapred.ShuffleHandler","yarn.sharedcache.store.in-memory.staleness-period-mins":"10080","fs.adl.impl":"org.apache.hadoop.fs.adl.AdlFileSystem","yarn.resourcemanager.application.max-tags":"10","hadoop.domainname.resolver.impl":"org.apache.hadoop.net.DNSDomainNameResolver","yarn.resourcemanager.nodemanager.minimum.version":"NONE","mapreduce.jobhistory.webapp.xfs-filter.xframe-options":"SAMEORIGIN","yarn.app.mapreduce.am.staging-dir.erasurecoding.enabled":"false","net.topology.impl":"org.apache.hadoop.net.NetworkTopology","io.map.index.skip":"0","yarn.timeline-service.reader.webapp.https.address":"${yarn.timeline-service.webapp.https.address}","fs.ftp.data.connection.mode":"ACTIVE_LOCAL_DATA_CONNECTION_MODE","mapreduce.job.local-fs.single-disk-limit.check.kill-limit-exceed":"true","fs.azure.buffer.dir":"${hadoop.tmp.dir}/abfs","yarn.scheduler.maximum-allocation-vcores":"4","hadoop.http.cross-origin.allowed-headers":"X-Requested-With,Content-Type,Accept,Origin","yarn.nodemanager.log-aggregation.compression-type":"none","yarn.timeline-service.version":"1.0f","yarn.ipc.rpc.class":"org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC","mapreduce.reduce.maxattempts":"4","yarn.resourcemanager.system-metrics-publisher.timeline-server-v1.batch-size":"1000","hadoop.security.dns.log-slow-lookups.enabled":"false","mapreduce.job.committer.setup.cleanup.needed":"true","hadoop.security.secure.random.impl":"org.apache.hadoop.crypto.random.OpensslSecureRandom","mapreduce.job.running.reduce.limit":"0","fs.s3a.select.errors.include.sql":"false","fs.s3a.connection.request.timeout":"0","ipc.maximum.response.length":"134217728","yarn.resourcemanager.webapp.rest-csrf.methods-to-ignore":"GET,OPTIONS,HEAD","mapreduce.job.token.tracking.ids.enabled":"*********(redacted)","hadoop.caller.context.max.size":"128","yarn.nodemanager.runtime.linux.docker.host-pid-namespace.allowed":"false","yarn.nodemanager.runtime.linux.docker.delayed-removal.allowed":"false","hadoop.registry.system.acls":"sasl:yarn@, sasl:mapred@, sasl:hdfs@","yarn.nodemanager.recovery.dir":"${hadoop.tmp.dir}/yarn-nm-recovery","fs.s3a.fast.upload.buffer":"disk","mapreduce.jobhistory.intermediate-done-dir":"${yarn.app.mapreduce.am.staging-dir}/history/done_intermediate","yarn.app.mapreduce.shuffle.log.separate":"true","yarn.log-aggregation.debug.filesize":"104857600","fs.s3a.max.total.tasks":"32","fs.s3a.readahead.range":"64K","hadoop.http.authentication.simple.anonymous.allowed":"true","fs.s3a.attempts.maximum":"20","hadoop.registry.zk.connection.timeout.ms":"15000","yarn.resourcemanager.delegation-token-renewer.thread-count":"*********(redacted)","yarn.resourcemanager.delegation-token-renewer.thread-timeout":"*********(redacted)","yarn.timeline-service.leveldb-timeline-store.start-time-write-cache-size":"10000","yarn.nodemanager.aux-services.manifest.reload-ms":"0","yarn.nodemanager.emit-container-events":"true","yarn.resourcemanager.resource-profiles.enabled":"false","yarn.timeline-service.hbase-schema.prefix":"prod.","fs.azure.authorization":"false","mapreduce.map.log.level":"INFO","ha.failover-controller.active-standby-elector.zk.op.retries":"3","yarn.resourcemanager.decommissioning-nodes-watcher.poll-interval-secs":"20","mapreduce.output.fileoutputformat.compress.type":"RECORD","yarn.resourcemanager.leveldb-state-store.path":"${hadoop.tmp.dir}/yarn/system/rmstore","yarn.timeline-service.webapp.rest-csrf.custom-header":"X-XSRF-Header","mapreduce.ifile.readahead.bytes":"4194304","yarn.sharedcache.app-checker.class":"org.apache.hadoop.yarn.server.sharedcachemanager.RemoteAppChecker","yarn.nodemanager.linux-container-executor.nonsecure-mode.limit-users":"true","yarn.nodemanager.resource.detect-hardware-capabilities":"false","mapreduce.cluster.acls.enabled":"false","mapreduce.job.speculative.retry-after-no-speculate":"1000","fs.viewfs.overload.scheme.target.abfs.impl":"org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem","hadoop.security.group.mapping.ldap.search.group.hierarchy.levels":"0","yarn.resourcemanager.fs.state-store.retry-interval-ms":"1000","file.stream-buffer-size":"4096","yarn.resourcemanager.application-timeouts.monitor.interval-ms":"3000","mapreduce.map.output.compress.codec":"org.apache.hadoop.io.compress.DefaultCodec","mapreduce.map.speculative":"true","yarn.nodemanager.runtime.linux.runc.image-tag-to-manifest-plugin.hdfs-hash-file":"/runc-root/image-tag-to-hash","mapreduce.job.speculative.retry-after-speculate":"15000","yarn.nodemanager.linux-container-executor.cgroups.mount":"false","yarn.app.mapreduce.am.container.log.backups":"0","yarn.app.mapreduce.am.log.level":"INFO","yarn.nodemanager.runtime.linux.runc.image-tag-to-manifest-plugin":"org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.runc.ImageTagToManifestPlugin","io.bytes.per.checksum":"512","mapreduce.job.reduce.slowstart.completedmaps":"0.05","yarn.timeline-service.http-authentication.type":"simple","hadoop.security.group.mapping.ldap.search.attr.group.name":"cn","yarn.nodemanager.resource-plugins.fpga.allowed-fpga-devices":"auto","yarn.timeline-service.client.internal-timers-ttl-secs":"420","fs.s3a.select.output.csv.quote.character":"\"","hadoop.http.logs.enabled":"true","fs.s3a.block.size":"32M","yarn.sharedcache.client-server.address":"0.0.0.0:8045","yarn.nodemanager.logaggregation.threadpool-size-max":"100","yarn.resourcemanager.hostname":"0.0.0.0","yarn.resourcemanager.delegation.key.update-interval":"86400000","mapreduce.reduce.shuffle.fetch.retry.enabled":"${yarn.nodemanager.recovery.enabled}","mapreduce.map.memory.mb":"-1","mapreduce.task.skip.start.attempts":"2","fs.AbstractFileSystem.hdfs.impl":"org.apache.hadoop.fs.Hdfs","yarn.nodemanager.disk-health-checker.enable":"true","fs.s3a.select.output.csv.quote.fields":"always","ipc.client.tcpnodelay":"true","ipc.client.rpc-timeout.ms":"0","yarn.nodemanager.webapp.rest-csrf.methods-to-ignore":"GET,OPTIONS,HEAD","yarn.resourcemanager.delegation-token-renewer.thread-retry-max-attempts":"*********(redacted)","ipc.client.low-latency":"false","mapreduce.input.lineinputformat.linespermap":"1","yarn.router.interceptor.user.threadpool-size":"5","ipc.client.connect.max.retries.on.timeouts":"45","yarn.timeline-service.leveldb-timeline-store.read-cache-size":"104857600","fs.AbstractFileSystem.har.impl":"org.apache.hadoop.fs.HarFs","mapreduce.job.split.metainfo.maxsize":"10000000","yarn.am.liveness-monitor.expiry-interval-ms":"600000","yarn.resourcemanager.container-tokens.master-key-rolling-interval-secs":"*********(redacted)","yarn.timeline-service.entity-group-fs-store.app-cache-size":"10","yarn.nodemanager.runtime.linux.runc.hdfs-manifest-to-resources-plugin.stat-cache-timeout-interval-secs":"360","fs.s3a.socket.recv.buffer":"8192","rpc.metrics.timeunit":"MILLISECONDS","yarn.resourcemanager.resource-tracker.address":"${yarn.resourcemanager.hostname}:8031","yarn.nodemanager.node-labels.provider.fetch-timeout-ms":"1200000","mapreduce.job.heap.memory-mb.ratio":"0.8","yarn.resourcemanager.leveldb-state-store.compaction-interval-secs":"3600","yarn.resourcemanager.webapp.rest-csrf.custom-header":"X-XSRF-Header","yarn.nodemanager.pluggable-device-framework.enabled":"false","yarn.scheduler.configuration.fs.path":"file://${hadoop.tmp.dir}/yarn/system/schedconf","mapreduce.client.output.filter":"FAILED","hadoop.http.filter.initializers":"org.apache.hadoop.http.lib.StaticUserWebFilter","mapreduce.reduce.memory.mb":"-1","yarn.timeline-service.hostname":"0.0.0.0","file.replication":"1","yarn.nodemanager.container-metrics.unregister-delay-ms":"10000","yarn.nodemanager.container-metrics.period-ms":"-1","mapreduce.fileoutputcommitter.task.cleanup.enabled":"false","yarn.nodemanager.log.retain-seconds":"10800","yarn.timeline-service.entity-group-fs-store.cleaner-interval-seconds":"3600","ipc.[port_number].callqueue.impl":"java.util.concurrent.LinkedBlockingQueue","yarn.resourcemanager.keytab":"/etc/krb5.keytab","hadoop.security.group.mapping.providers.combined":"true","mapreduce.reduce.merge.inmem.threshold":"1000","yarn.timeline-service.recovery.enabled":"false","fs.azure.saskey.usecontainersaskeyforallaccess":"true","yarn.sharedcache.nm.uploader.thread-count":"20","yarn.resourcemanager.nodemanager-graceful-decommission-timeout-secs":"3600","ipc.[port_number].weighted-cost.lockfree":"1","mapreduce.shuffle.ssl.enabled":"false","yarn.timeline-service.hbase.coprocessor.app-final-value-retention-milliseconds":"259200000","yarn.nodemanager.opportunistic-containers-max-queue-length":"0","yarn.resourcemanager.state-store.max-completed-applications":"${yarn.resourcemanager.max-completed-applications}","mapreduce.job.speculative.minimum-allowed-tasks":"10","fs.s3a.aws.credentials.provider":"\n org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider,\n org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider,\n com.amazonaws.auth.EnvironmentVariableCredentialsProvider,\n org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider\n ","yarn.log-aggregation.retain-seconds":"-1","yarn.nodemanager.disk-health-checker.min-free-space-per-disk-mb":"0","mapreduce.jobhistory.max-age-ms":"604800000","hadoop.http.cross-origin.allowed-methods":"GET,POST,HEAD","yarn.resourcemanager.opportunistic-container-allocation.enabled":"false","mapreduce.jobhistory.webapp.address":"0.0.0.0:19888","hadoop.system.tags":"YARN,HDFS,NAMENODE,DATANODE,REQUIRED,SECURITY,KERBEROS,PERFORMANCE,CLIENT\n ,SERVER,DEBUG,DEPRECATED,COMMON,OPTIONAL","yarn.log-aggregation.file-controller.TFile.class":"org.apache.hadoop.yarn.logaggregation.filecontroller.tfile.LogAggregationTFileController","yarn.client.nodemanager-connect.max-wait-ms":"180000","yarn.resourcemanager.webapp.address":"${yarn.resourcemanager.hostname}:8088","mapreduce.jobhistory.recovery.enable":"false","mapreduce.reduce.shuffle.parallelcopies":"5","fs.AbstractFileSystem.webhdfs.impl":"org.apache.hadoop.fs.WebHdfs","fs.trash.interval":"0","yarn.app.mapreduce.client.max-retries":"3","hadoop.security.authentication":"simple","mapreduce.task.profile.reduce.params":"${mapreduce.task.profile.params}","yarn.app.mapreduce.am.resource.mb":"1536","mapreduce.input.fileinputformat.list-status.num-threads":"1","yarn.nodemanager.container-executor.class":"org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor","io.mapfile.bloom.size":"1048576","yarn.timeline-service.ttl-ms":"604800000","yarn.resourcemanager.nm-container-queuing.min-queue-length":"5","yarn.nodemanager.resource.cpu-vcores":"-1","mapreduce.job.reduces":"1","fs.s3a.multipart.size":"64M","fs.s3a.select.input.csv.comment.marker":"#","yarn.scheduler.minimum-allocation-vcores":"1","mapreduce.job.speculative.speculative-cap-total-tasks":"0.01","hadoop.ssl.client.conf":"ssl-client.xml","mapreduce.job.queuename":"default","mapreduce.job.encrypted-intermediate-data-key-size-bits":"128","fs.s3a.metadatastore.authoritative":"false","ipc.[port_number].weighted-cost.response":"1","yarn.nodemanager.webapp.xfs-filter.xframe-options":"SAMEORIGIN","ha.health-monitor.sleep-after-disconnect.ms":"1000","yarn.app.mapreduce.shuffle.log.limit.kb":"0","hadoop.security.group.mapping":"org.apache.hadoop.security.JniBasedUnixGroupsMappingWithFallback","yarn.client.application-client-protocol.poll-timeout-ms":"-1","mapreduce.jobhistory.jhist.format":"binary","mapreduce.task.stuck.timeout-ms":"600000","yarn.resourcemanager.application.max-tag.length":"100","yarn.resourcemanager.ha.enabled":"false","dfs.client.ignore.namenode.default.kms.uri":"false","hadoop.http.staticuser.user":"dr.who","mapreduce.task.exit.timeout.check-interval-ms":"20000","mapreduce.jobhistory.intermediate-user-done-dir.permissions":"770","mapreduce.task.exit.timeout":"60000","yarn.nodemanager.linux-container-executor.resources-handler.class":"org.apache.hadoop.yarn.server.nodemanager.util.DefaultLCEResourcesHandler","mapreduce.reduce.shuffle.memory.limit.percent":"0.25","yarn.resourcemanager.reservation-system.enable":"false","mapreduce.map.output.compress":"false","ha.zookeeper.acl":"world:anyone:rwcda","ipc.server.max.connections":"0","yarn.nodemanager.runtime.linux.docker.default-container-network":"host","yarn.router.webapp.address":"0.0.0.0:8089","yarn.scheduler.maximum-allocation-mb":"8192","yarn.resourcemanager.scheduler.monitor.policies":"org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy","yarn.sharedcache.cleaner.period-mins":"1440","yarn.nodemanager.resource-plugins.gpu.docker-plugin.nvidia-docker-v1.endpoint":"http://localhost:3476/v1.0/docker/cli","yarn.app.mapreduce.am.container.log.limit.kb":"0","ipc.client.connect.retry.interval":"1000","yarn.timeline-service.http-cross-origin.enabled":"false","fs.wasbs.impl":"org.apache.hadoop.fs.azure.NativeAzureFileSystem$Secure","yarn.resourcemanager.nodemanagers.heartbeat-interval-max-ms":"1000","yarn.federation.subcluster-resolver.class":"org.apache.hadoop.yarn.server.federation.resolver.DefaultSubClusterResolverImpl","yarn.resourcemanager.zk-state-store.parent-path":"/rmstore","fs.s3a.select.input.csv.field.delimiter":",","mapreduce.jobhistory.cleaner.enable":"true","yarn.timeline-service.client.fd-flush-interval-secs":"10","hadoop.security.kms.client.encrypted.key.cache.expiry":"43200000","yarn.client.nodemanager-client-async.thread-pool-max-size":"500","mapreduce.map.maxattempts":"4","yarn.resourcemanager.nm-container-queuing.sorting-nodes-interval-ms":"1000","fs.s3a.committer.staging.tmp.path":"tmp/staging","yarn.nodemanager.sleep-delay-before-sigkill.ms":"250","yarn.resourcemanager.nm-container-queuing.min-queue-wait-time-ms":"10","mapreduce.job.end-notification.retry.attempts":"0","yarn.nodemanager.resource.count-logical-processors-as-cores":"false","hadoop.registry.zk.root":"/registry","adl.feature.ownerandgroup.enableupn":"false","yarn.resourcemanager.zk-max-znode-size.bytes":"1048576","mapreduce.job.reduce.shuffle.consumer.plugin.class":"org.apache.hadoop.mapreduce.task.reduce.Shuffle","yarn.resourcemanager.delayed.delegation-token.removal-interval-ms":"*********(redacted)","yarn.nodemanager.localizer.cache.target-size-mb":"10240","fs.s3a.committer.staging.conflict-mode":"append","mapreduce.client.libjars.wildcard":"true","fs.s3a.committer.staging.unique-filenames":"true","yarn.nodemanager.node-attributes.provider.fetch-timeout-ms":"1200000","fs.s3a.list.version":"2","ftp.client-write-packet-size":"65536","ipc.[port_number].weighted-cost.lockexclusive":"100","fs.AbstractFileSystem.adl.impl":"org.apache.hadoop.fs.adl.Adl","yarn.nodemanager.container-log-monitor.enable":"false","hadoop.security.key.default.cipher":"AES/CTR/NoPadding","yarn.client.failover-retries":"0","fs.s3a.multipart.purge.age":"86400","mapreduce.job.local-fs.single-disk-limit.check.interval-ms":"5000","net.topology.node.switch.mapping.impl":"org.apache.hadoop.net.ScriptBasedMapping","yarn.nodemanager.amrmproxy.address":"0.0.0.0:8049","ipc.server.listen.queue.size":"256","ipc.[port_number].decay-scheduler.period-ms":"5000","yarn.nodemanager.runtime.linux.runc.image-tag-to-manifest-plugin.cache-refresh-interval-secs":"60","map.sort.class":"org.apache.hadoop.util.QuickSort","fs.viewfs.rename.strategy":"SAME_MOUNTPOINT","hadoop.security.kms.client.authentication.retry-count":"1","fs.permissions.umask-mode":"022","fs.s3a.assumed.role.credentials.provider":"org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider","yarn.nodemanager.runtime.linux.runc.privileged-containers.allowed":"false","yarn.nodemanager.vmem-check-enabled":"true","yarn.nodemanager.numa-awareness.enabled":"false","yarn.nodemanager.recovery.compaction-interval-secs":"3600","yarn.app.mapreduce.client-am.ipc.max-retries":"3","yarn.resourcemanager.system-metrics-publisher.timeline-server-v1.interval-seconds":"60","yarn.federation.registry.base-dir":"yarnfederation/","yarn.nodemanager.health-checker.run-before-startup":"false","mapreduce.job.max.map":"-1","mapreduce.job.local-fs.single-disk-limit.bytes":"-1","mapreduce.shuffle.pathcache.concurrency-level":"16","mapreduce.job.ubertask.maxreduces":"1","mapreduce.shuffle.pathcache.max-weight":"10485760","hadoop.security.kms.client.encrypted.key.cache.size":"500","hadoop.security.java.secure.random.algorithm":"SHA1PRNG","ha.failover-controller.cli-check.rpc-timeout.ms":"20000","mapreduce.jobhistory.jobname.limit":"50","fs.s3a.select.input.compression":"none","yarn.client.nodemanager-connect.retry-interval-ms":"10000","ipc.[port_number].scheduler.priority.levels":"4","yarn.timeline-service.state-store-class":"org.apache.hadoop.yarn.server.timeline.recovery.LeveldbTimelineStateStore","yarn.nodemanager.env-whitelist":"JAVA_HOME,HADOOP_COMMON_HOME,HADOOP_HDFS_HOME,HADOOP_CONF_DIR,CLASSPATH_PREPEND_DISTCACHE,HADOOP_YARN_HOME,HADOOP_HOME,PATH,LANG,TZ","yarn.sharedcache.nested-level":"3","yarn.timeline-service.webapp.rest-csrf.methods-to-ignore":"GET,OPTIONS,HEAD","fs.azure.user.agent.prefix":"unknown","yarn.resourcemanager.zk-delegation-token-node.split-index":"*********(redacted)","yarn.nodemanager.numa-awareness.read-topology":"false","yarn.nodemanager.webapp.address":"${yarn.nodemanager.hostname}:8042","rpc.metrics.quantile.enable":"false","yarn.registry.class":"org.apache.hadoop.registry.client.impl.FSRegistryOperationsService","mapreduce.jobhistory.admin.acl":"*","yarn.resourcemanager.system-metrics-publisher.dispatcher.pool-size":"10","yarn.scheduler.queue-placement-rules":"user-group","hadoop.http.authentication.kerberos.keytab":"${user.home}/hadoop.keytab","yarn.resourcemanager.recovery.enabled":"false","fs.s3a.select.input.csv.header":"none","yarn.nodemanager.runtime.linux.runc.hdfs-manifest-to-resources-plugin.stat-cache-size":"500","yarn.timeline-service.webapp.rest-csrf.enabled":"false","yarn.nodemanager.disk-health-checker.min-free-space-per-disk-watermark-high-mb":"0"},"System Properties":{"java.io.tmpdir":"/tmp","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"17","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Private Build","java.vm.specification.version":"17","user.home":"/home/holden","sun.arch.data.model":"64","sun.boot.library.path":"/usr/lib/jvm/java-17-openjdk-amd64/lib","user.dir":"/home/holden/repos/sparklens","java.library.path":"/usr/java/packages/lib:/usr/lib/x86_64-linux-gnu/jni:/lib/x86_64-linux-gnu:/usr/lib/x86_64-linux-gnu:/usr/lib/jni:/lib:/usr/lib","os.arch":"amd64","java.vm.version":"17.0.10+7-Ubuntu-122.04.1","jetty.git.hash":"6b67c5719d1f4371b33655ff2d047d24e171e49a","java.runtime.version":"17.0.10+7-Ubuntu-122.04.1","java.vm.info":"mixed mode, sharing","java.runtime.name":"OpenJDK Runtime Environment","java.version.date":"2024-01-16","file.separator":"/","java.class.version":"61.0","java.specification.name":"Java Platform API Specification","file.encoding":"UTF-8","user.timezone":"America/Los_Angeles","kubernetes.request.retry.backoffLimit":"3","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","java.vm.compressedOopsMode":"32-bit","os.version":"6.5.0-28-generic","native.encoding":"UTF-8","java.vm.specification.vendor":"Oracle Corporation","sun.stderr.encoding":"UTF-8","user.country":"US","sun.jnu.encoding":"UTF-8","user.language":"en","sun.stdout.encoding":"UTF-8","java.vendor.url":"Unknown","os.name":"Linux","java.vm.vendor":"Private Build","jdk.debug":"release","java.vendor.url.bug":"Unknown","user.name":"holden","java.vm.name":"OpenJDK 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master k8s://https://127.0.0.1:16443 --deploy-mode client --conf spark.dynamicAllocation.shuffleTracking.enabled=true --conf spark.sql.catalog.spark_catalog.type=hive --conf spark.kubernetes.container.image=localhost:32000/local-spark/spark:magic --conf spark.kubernetes.authenticate.submission.caCertFile=/var/snap/microk8s/current/certs/ca.crt --conf spark.sql.catalog.spark_catalog=org.apache.iceberg.spark.SparkSessionCatalog --conf spark.sql.catalog.local=org.apache.iceberg.spark.SparkCatalog --conf spark.eventLog.enabled=true --conf spark.sql.catalog.local.warehouse=/tmp/warehouse --conf spark.kubernetes.authenticate.caCertFile=/var/snap/microk8s/current/certs/ca.crt --conf spark.dynamicAllocation.enabled=true --conf spark.sql.catalog.local.type=hadoop --class org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver --name fresh-kube spark-internal -f e2e/partioned_table_join.sql","java.home":"/usr/lib/jvm/java-17-openjdk-amd64","java.version":"17.0.10","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/httpclient-4.5.13.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jodd-core-3.5.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/JLargeArrays-1.5.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/scala-library-2.12.15.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/javassist-3.25.0-GA.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/commons-lang3-3.12.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/protobuf-java-2.5.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/compress-lzf-1.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hive-shims-common-2.3.9.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-network-shuffle_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hadoop-client-runtime-3.3.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-discovery-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/commons-io-2.11.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/breeze_2.12-1.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/snappy-java-1.1.8.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jakarta.inject-2.6.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spire-platform_2.12-0.17.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jakarta.annotation-api-1.3.5.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/commons-compress-1.21.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hive-beeline-2.3.9.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/libthrift-0.12.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/metrics-json-4.2.7.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-kvstore_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/netty-transport-native-epoll-4.1.74.Final-linux-aarch_64.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/scala-collection-compat_2.12-2.1.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/netty-handler-4.1.74.Final.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/parquet-encoding-1.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/scala-xml_2.12-1.2.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/log4j-1.2-api-2.17.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-unsafe_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/javolution-5.5.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jakarta.servlet-api-4.0.3.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/netty-resolver-4.1.74.Final.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jakarta.validation-api-2.0.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/datanucleus-rdbms-4.1.19.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hive-common-2.3.9.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/cats-kernel_2.12-2.1.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/opencsv-2.3.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/janino-3.0.16.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/velocity-1.5.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/univocity-parsers-2.9.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jersey-server-2.36.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-events-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-network-common_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/zjsonpatch-0.3.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/dropwizard-metrics-hadoop-metrics2-reporter-0.1.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-admissionregistration-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/json4s-ast_2.12-3.7.0-M11.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/transaction-api-1.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jackson-datatype-jsr310-2.13.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-hive_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hadoop-client-api-3.3.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-mesos_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/json4s-jackson_2.12-3.7.0-M11.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/logging-interceptor-3.12.12.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jackson-databind-2.13.4.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-sketch_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/guava-14.0.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/pickle-1.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-streaming_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-apiextensions-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/metrics-jmx-4.2.7.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-storageclass-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/leveldbjni-all-1.8.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/audience-annotations-0.5.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/commons-crypto-1.1.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/parquet-format-structures-1.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/netty-buffer-4.1.74.Final.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spire-util_2.12-0.17.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/iceberg-spark-runtime-3.3_2.12-1.4.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jline-2.14.6.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/HikariCP-2.5.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/ST4-4.0.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/bonecp-0.8.0.RELEASE.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/lz4-java-1.8.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jersey-container-servlet-2.36.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/arpack-2.2.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/commons-logging-1.1.3.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hive-service-rpc-3.1.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/shims-0.9.25.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/commons-codec-1.15.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-metrics-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jackson-dataformat-yaml-2.13.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/commons-compiler-3.0.16.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/commons-collections-3.2.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/chill_2.12-0.10.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/arrow-vector-7.0.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kryo-shaded-4.0.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/curator-recipes-2.13.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-tags_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jcl-over-slf4j-1.7.32.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/scala-reflect-2.12.15.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/metrics-graphite-4.2.7.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/arrow-memory-netty-7.0.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hive-shims-0.23-2.3.9.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/parquet-hadoop-1.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spire-macros_2.12-0.17.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-tags_2.12-3.3.4-tests.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/bcprov-jdk15on-1.68.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/conf/":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jackson-core-2.13.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/json4s-scalap_2.12-3.7.0-M11.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/json-1.8.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/derby-10.14.2.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-core-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jdo-api-3.0.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/scala-compiler-2.12.15.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-repl_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jaxb-runtime-2.3.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jersey-client-2.36.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/JTransforms-3.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-client-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/breeze-macros_2.12-1.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hadoop-shaded-guava-1.1.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/json4s-core_2.12-3.7.0-M11.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/arrow-format-7.0.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hive-vector-code-gen-2.3.9.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/okhttp-3.12.12.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/xz-1.9.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hive-jdbc-2.3.9.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hive-exec-2.3.9-core.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/slf4j-api-1.7.32.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-scheduling-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/aopalliance-repackaged-2.6.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-coordination-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-rbac-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/datanucleus-core-4.1.17.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/RoaringBitmap-0.9.25.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/core-1.1.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jsr305-3.0.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/parquet-common-1.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/antlr-runtime-3.5.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/commons-math3-3.6.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/aircompressor-0.21.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hadoop-yarn-server-web-proxy-3.3.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/orc-core-1.7.10.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/orc-shims-1.7.10.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/arpack_combined_all-0.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-catalyst_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/generex-1.0.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/netty-transport-native-kqueue-4.1.74.Final-osx-x86_64.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/py4j-0.10.9.5.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jakarta.ws.rs-api-2.1.6.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/commons-text-1.10.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-node-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-extensions-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/netty-all-4.1.74.Final.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/ivy-2.5.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/flatbuffers-java-1.12.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/arrow-memory-core-7.0.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hive-cli-2.3.9.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/netty-tcnative-classes-2.0.48.Final.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/bcpkix-jdk15on-1.68.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/netty-transport-native-epoll-4.1.74.Final-linux-x86_64.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/httpcore-4.4.14.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/log4j-core-2.17.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/commons-dbcp-1.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/netty-transport-classes-epoll-4.1.74.Final.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/osgi-resource-locator-1.0.3.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/curator-client-2.13.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/avro-ipc-1.11.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/stream-2.9.6.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-hive-thriftserver_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/datanucleus-api-jdo-4.2.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/okio-1.14.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/rocksdbjni-6.20.3.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-flowcontrol-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/netty-transport-native-unix-common-4.1.74.Final.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-launcher_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jul-to-slf4j-1.7.32.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-certificates-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/netty-common-4.1.74.Final.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/algebra_2.12-2.0.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-mllib_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jpam-1.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/avro-1.11.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/avro-mapred-1.11.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/orc-mapreduce-1.7.10.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-batch-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/commons-cli-1.5.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-kubernetes_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jackson-module-scala_2.12-2.13.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/scala-parser-combinators_2.12-1.1.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/gson-2.2.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spire_2.12-0.17.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/zookeeper-jute-3.6.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/oro-2.0.8.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hk2-utils-2.6.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-core_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/blas-2.2.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/annotations-17.0.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/log4j-api-2.17.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/tink-1.6.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/chill-java-0.10.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hive-storage-api-2.7.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-networking-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/curator-framework-2.13.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-mllib-local_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/zookeeper-3.6.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/commons-collections4-4.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/minlog-1.3.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hive-serde-2.3.9.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/snakeyaml-1.31.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/lapack-2.2.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/commons-lang-2.6.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jersey-common-2.36.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jersey-hk2-2.36.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/parquet-jackson-1.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-yarn_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/istack-commons-runtime-3.0.8.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-common-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/automaton-1.11-8.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-sql_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/antlr4-runtime-4.8.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/netty-codec-4.1.74.Final.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jta-1.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jakarta.xml.bind-api-2.3.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/metrics-jvm-4.2.7.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/shapeless_2.12-2.3.7.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-apps-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/spark-graphx_2.12-3.3.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/log4j-slf4j-impl-2.17.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/paranamer-2.8.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/xbean-asm9-shaded-4.20.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jackson-annotations-2.13.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/activation-1.1.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hk2-locator-2.6.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hk2-api-2.6.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-policy-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/netty-transport-native-kqueue-4.1.74.Final-osx-aarch_64.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/netty-transport-4.1.74.Final.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/mesos-1.4.3-shaded-protobuf.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/super-csv-2.2.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/joda-time-2.10.13.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/objenesis-3.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hive-shims-scheduler-2.3.9.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/javax.jdo-3.2.0-m3.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hive-shims-2.3.9.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/stax-api-1.0.1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/metrics-core-4.2.7.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/zstd-jni-1.5.2-1.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/libfb303-0.9.3.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/netty-transport-classes-kqueue-4.1.74.Final.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/commons-pool-1.5.4.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/kubernetes-model-autoscaling-5.12.2.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hive-llap-common-2.3.9.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/hive-metastore-2.3.9.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/jersey-container-servlet-core-2.36.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/threeten-extra-1.5.0.jar":"System Classpath","/home/holden/repos/sparklens/spark-3.3.4-bin-hadoop3/jars/parquet-column-1.12.2.jar":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"fresh-kube","App ID":"spark-a8d39974ec7b451a92b303fa953434e4","Timestamp":1714515304235,"User":"holden"} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":0,"description":"CREATE TABLE IF NOT EXISTS local.developers (\n username string,\n firstname string,\n lastname string)\nUSING iceberg\nPARTITIONED BY (username)","details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nCreateTable (1)\n\n\n(1) CreateTable\nArguments: org.apache.iceberg.spark.SparkCatalog@41cfcbb5, developers, [StructField(username,StringType,true), StructField(firstname,StringType,true), StructField(lastname,StringType,true)], [identity(username)], TableSpec(Map(),Some(iceberg),Map(),None,None,None,false), true\n\n","sparkPlanInfo":{"nodeName":"CreateTable","simpleString":"CreateTable org.apache.iceberg.spark.SparkCatalog@41cfcbb5, developers, [StructField(username,StringType,true), StructField(firstname,StringType,true), StructField(lastname,StringType,true)], [identity(username)], TableSpec(Map(),Some(iceberg),Map(),None,None,None,false), true","children":[],"metadata":{},"metrics":[]},"time":1714515307749,"modifiedConfigs":{}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":0,"time":1714515307929} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":1,"description":"CREATE TABLE IF NOT EXISTS local.developers (\n username string,\n firstname string,\n lastname string)\nUSING iceberg\nPARTITIONED BY (username)","details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nCommandResult (1)\n +- CreateTable (2)\n\n\n(1) CommandResult\nOutput: []\nArguments: \n\n(2) CreateTable\nArguments: org.apache.iceberg.spark.SparkCatalog@41cfcbb5, developers, [StructField(username,StringType,true), StructField(firstname,StringType,true), StructField(lastname,StringType,true)], [identity(username)], TableSpec(Map(),Some(iceberg),Map(),None,None,None,false), true\n\n","sparkPlanInfo":{"nodeName":"CommandResult","simpleString":"CommandResult ","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":0,"metricType":"sum"}]},"time":1714515307948,"modifiedConfigs":{}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":1,"time":1714515307965} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":2,"description":"\nCREATE TABLE IF NOT EXISTS local.projects (\n creator string,\n projectname string)\nUSING iceberg\nPARTITIONED BY (creator)","details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nCreateTable (1)\n\n\n(1) CreateTable\nArguments: org.apache.iceberg.spark.SparkCatalog@41cfcbb5, projects, [StructField(creator,StringType,true), StructField(projectname,StringType,true)], [identity(creator)], TableSpec(Map(),Some(iceberg),Map(),None,None,None,false), true\n\n","sparkPlanInfo":{"nodeName":"CreateTable","simpleString":"CreateTable org.apache.iceberg.spark.SparkCatalog@41cfcbb5, projects, [StructField(creator,StringType,true), StructField(projectname,StringType,true)], [identity(creator)], TableSpec(Map(),Some(iceberg),Map(),None,None,None,false), true","children":[],"metadata":{},"metrics":[]},"time":1714515307986,"modifiedConfigs":{}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":2,"time":1714515308009} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":3,"description":"\nCREATE TABLE IF NOT EXISTS local.projects (\n creator string,\n projectname string)\nUSING iceberg\nPARTITIONED BY (creator)","details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nCommandResult (1)\n +- CreateTable (2)\n\n\n(1) CommandResult\nOutput: []\nArguments: \n\n(2) CreateTable\nArguments: org.apache.iceberg.spark.SparkCatalog@41cfcbb5, projects, [StructField(creator,StringType,true), StructField(projectname,StringType,true)], [identity(creator)], TableSpec(Map(),Some(iceberg),Map(),None,None,None,false), true\n\n","sparkPlanInfo":{"nodeName":"CommandResult","simpleString":"CommandResult ","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1,"metricType":"sum"}]},"time":1714515308017,"modifiedConfigs":{}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":3,"time":1714515308017} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":4,"description":"\nINSERT INTO local.developers VALUES(\"krisnova\", \"Kris\", \"Nova\")","details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nAppendData (4)\n+- AdaptiveSparkPlan (3)\n +- Exchange (2)\n +- LocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [3]: [username#6, firstname#7, lastname#8]\nArguments: [username#6, firstname#7, lastname#8]\n\n(2) Exchange\nInput [3]: [username#6, firstname#7, lastname#8]\nArguments: hashpartitioning(username#6, 200), REPARTITION_BY_NUM, [plan_id=21]\n\n(3) AdaptiveSparkPlan\nOutput [3]: [username#6, firstname#7, lastname#8]\nArguments: isFinalPlan=false\n\n(4) AppendData\nInput [3]: [username#6, firstname#7, lastname#8]\nArguments: org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$2453/0x00007505ed0efbf0@418f2af3, IcebergWrite(table=local.developers, format=PARQUET)\n\n","sparkPlanInfo":{"nodeName":"AppendData","simpleString":"AppendData org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$2453/0x00007505ed0efbf0@418f2af3, IcebergWrite(table=local.developers, format=PARQUET)","children":[{"nodeName":"AdaptiveSparkPlan","simpleString":"AdaptiveSparkPlan isFinalPlan=false","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(username#6, 200), REPARTITION_BY_NUM, [plan_id=21]","children":[{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [username#6, firstname#7, lastname#8]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":14,"metricType":"sum"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":12,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":13,"metricType":"nsTiming"},{"name":"records read","accumulatorId":10,"metricType":"sum"},{"name":"local bytes read","accumulatorId":8,"metricType":"size"},{"name":"fetch wait time","accumulatorId":9,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":6,"metricType":"size"},{"name":"local blocks read","accumulatorId":5,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":4,"metricType":"sum"},{"name":"data size","accumulatorId":2,"metricType":"size"},{"name":"number of partitions","accumulatorId":3,"metricType":"sum"},{"name":"remote bytes read to disk","accumulatorId":7,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":11,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]},"time":1714515308155,"modifiedConfigs":{}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate","executionId":4,"physicalPlanDescription":"== Physical Plan ==\nAppendData (5)\n+- AdaptiveSparkPlan (4)\n +- == Current Plan ==\n ShuffleQueryStage (3)\n +- Exchange (2)\n +- LocalTableScan (1)\n +- == Initial Plan ==\n Exchange (2)\n +- LocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [3]: [username#6, firstname#7, lastname#8]\nArguments: [username#6, firstname#7, lastname#8]\n\n(2) Exchange\nInput [3]: [username#6, firstname#7, lastname#8]\nArguments: hashpartitioning(username#6, 200), REPARTITION_BY_NUM, [plan_id=21]\n\n(3) ShuffleQueryStage\nOutput [3]: [username#6, firstname#7, lastname#8]\nArguments: 0\n\n(4) AdaptiveSparkPlan\nOutput [3]: [username#6, firstname#7, lastname#8]\nArguments: isFinalPlan=false\n\n(5) AppendData\nInput [3]: [username#6, firstname#7, lastname#8]\nArguments: org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$2453/0x00007505ed0efbf0@418f2af3, IcebergWrite(table=local.developers, format=PARQUET)\n\n","sparkPlanInfo":{"nodeName":"AppendData","simpleString":"AppendData org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$2453/0x00007505ed0efbf0@418f2af3, IcebergWrite(table=local.developers, format=PARQUET)","children":[{"nodeName":"AdaptiveSparkPlan","simpleString":"AdaptiveSparkPlan isFinalPlan=false","children":[{"nodeName":"ShuffleQueryStage","simpleString":"ShuffleQueryStage 0","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(username#6, 200), REPARTITION_BY_NUM, [plan_id=21]","children":[{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [username#6, firstname#7, lastname#8]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":14,"metricType":"sum"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":12,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":13,"metricType":"nsTiming"},{"name":"records read","accumulatorId":10,"metricType":"sum"},{"name":"local bytes read","accumulatorId":8,"metricType":"size"},{"name":"fetch wait time","accumulatorId":9,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":6,"metricType":"size"},{"name":"local blocks read","accumulatorId":5,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":4,"metricType":"sum"},{"name":"data size","accumulatorId":2,"metricType":"size"},{"name":"number of partitions","accumulatorId":3,"metricType":"sum"},{"name":"remote bytes read to disk","accumulatorId":7,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":11,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":4,"accumUpdates":[[3,200]]} +{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1714515308350,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"processLine at CliDriver.java:336","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"Exchange\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"LocalTableScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"2\",\"name\":\"LocalTableScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[0],"Properties":{"spark.dynamicAllocation.shuffleTracking.enabled":"true","spark.sql.warehouse.dir":"file:/home/holden/repos/sparklens/spark-warehouse","spark.executor.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.driver.host":"10.69.200.180","spark.eventLog.enabled":"true","spark.driver.port":"37323","spark.sql.catalog.local.type":"hadoop","spark.jars":"","spark.sql.catalog.spark_catalog":"org.apache.iceberg.spark.SparkSessionCatalog","spark.kubernetes.authenticate.submission.caCertFile":"/var/snap/microk8s/current/certs/ca.crt","spark.sql.datetime.java8API.enabled":"true","spark.sql.hive.version":"2.3.9","spark.app.name":"fresh-kube","spark.rdd.scope":"{\"id\":\"1\",\"name\":\"Exchange\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.submitTime":"1714515303940","spark.job.description":"\nINSERT INTO local.developers VALUES(\"krisnova\", \"Kris\", \"Nova\")","spark.kubernetes.container.image":"localhost:32000/local-spark/spark:magic","spark.app.startTime":"1714515304235","spark.executor.id":"driver","spark.sql.catalog.local.warehouse":"/tmp/warehouse","spark.driver.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.sql.catalog.local":"org.apache.iceberg.spark.SparkCatalog","spark.submit.deployMode":"client","spark.master":"k8s://https://127.0.0.1:16443","spark.kubernetes.executor.podNamePrefix":"fresh-kube-bb0c5b8f311360be","spark.dynamicAllocation.enabled":"true","spark.sql.execution.id":"4","spark.sql.catalogImplementation":"hive","spark.kubernetes.authenticate.caCertFile":"/var/snap/microk8s/current/certs/ca.crt","spark.sql.catalog.spark_catalog.type":"hive","spark.app.id":"spark-a8d39974ec7b451a92b303fa953434e4"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"processLine at CliDriver.java:336","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"Exchange\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"LocalTableScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"2\",\"name\":\"LocalTableScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1714515308360,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.dynamicAllocation.shuffleTracking.enabled":"true","spark.sql.warehouse.dir":"file:/home/holden/repos/sparklens/spark-warehouse","spark.executor.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.driver.host":"10.69.200.180","spark.eventLog.enabled":"true","spark.driver.port":"37323","spark.sql.catalog.local.type":"hadoop","spark.jars":"","spark.sql.catalog.spark_catalog":"org.apache.iceberg.spark.SparkSessionCatalog","spark.kubernetes.authenticate.submission.caCertFile":"/var/snap/microk8s/current/certs/ca.crt","spark.sql.datetime.java8API.enabled":"true","spark.sql.hive.version":"2.3.9","spark.app.name":"fresh-kube","spark.rdd.scope":"{\"id\":\"1\",\"name\":\"Exchange\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.submitTime":"1714515303940","spark.job.description":"\nINSERT INTO local.developers VALUES(\"krisnova\", \"Kris\", \"Nova\")","spark.kubernetes.container.image":"localhost:32000/local-spark/spark:magic","spark.app.startTime":"1714515304235","spark.executor.id":"driver","spark.sql.catalog.local.warehouse":"/tmp/warehouse","spark.driver.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.sql.catalog.local":"org.apache.iceberg.spark.SparkCatalog","spark.submit.deployMode":"client","spark.master":"k8s://https://127.0.0.1:16443","spark.kubernetes.executor.podNamePrefix":"fresh-kube-bb0c5b8f311360be","spark.dynamicAllocation.enabled":"true","spark.sql.execution.id":"4","spark.sql.catalogImplementation":"hive","spark.kubernetes.authenticate.caCertFile":"/var/snap/microk8s/current/certs/ca.crt","spark.sql.catalog.spark_catalog.type":"hive","spark.app.id":"spark-a8d39974ec7b451a92b303fa953434e4"}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1714515311911,"Executor ID":"1","Executor Info":{"Host":"10.1.246.124","Total Cores":1,"Log Urls":{},"Attributes":{},"Resources":{},"Resource Profile Id":0}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"10.1.246.124","Port":35957},"Maximum Memory":434031820,"Timestamp":1714515311945,"Maximum Onheap Memory":434031820,"Maximum Offheap Memory":0} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Partition ID":0,"Launch Time":1714515311996,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Partition ID":0,"Launch Time":1714515311996,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515312523,"Failed":false,"Killed":false,"Accumulables":[{"ID":2,"Name":"data size","Update":"56","Value":"56","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":11,"Name":"shuffle bytes written","Update":"95","Value":"95","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":12,"Name":"shuffle records written","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":13,"Name":"shuffle write time","Update":"15182347","Value":"15182347","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":232,"Value":232,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":194901376,"Value":194901376,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":258,"Value":258,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":254903828,"Value":254903828,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1822,"Value":1822,"Internal":true,"Count Failed Values":true},{"ID":33,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":95,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":34,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":35,"Name":"internal.metrics.shuffle.write.writeTime","Update":15182347,"Value":15182347,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":232,"Executor Deserialize CPU Time":194901376,"Executor Run Time":258,"Executor CPU Time":254903828,"Peak Execution Memory":0,"Result Size":1822,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":95,"Shuffle Write Time":15182347,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"processLine at CliDriver.java:336","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"Exchange\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"LocalTableScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"2\",\"name\":\"LocalTableScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1714515308360,"Completion Time":1714515312528,"Accumulables":[{"ID":2,"Name":"data size","Value":"56","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":11,"Name":"shuffle bytes written","Value":"95","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":12,"Name":"shuffle records written","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":13,"Name":"shuffle write time","Value":"15182347","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Value":232,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Value":194901376,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Value":258,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Value":254903828,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Value":1822,"Internal":true,"Count Failed Values":true},{"ID":33,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":95,"Internal":true,"Count Failed Values":true},{"ID":34,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":1,"Internal":true,"Count Failed Values":true},{"ID":35,"Name":"internal.metrics.shuffle.write.writeTime","Value":15182347,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1714515312532,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate","executionId":4,"physicalPlanDescription":"== Physical Plan ==\nAppendData (5)\n+- AdaptiveSparkPlan (4)\n +- == Final Plan ==\n ShuffleQueryStage (3), Statistics(sizeInBytes=56.0 B, rowCount=1)\n +- Exchange (2)\n +- LocalTableScan (1)\n +- == Initial Plan ==\n Exchange (2)\n +- LocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [3]: [username#6, firstname#7, lastname#8]\nArguments: [username#6, firstname#7, lastname#8]\n\n(2) Exchange\nInput [3]: [username#6, firstname#7, lastname#8]\nArguments: hashpartitioning(username#6, 200), REPARTITION_BY_NUM, [plan_id=21]\n\n(3) ShuffleQueryStage\nOutput [3]: [username#6, firstname#7, lastname#8]\nArguments: 0\n\n(4) AdaptiveSparkPlan\nOutput [3]: [username#6, firstname#7, lastname#8]\nArguments: isFinalPlan=true\n\n(5) AppendData\nInput [3]: [username#6, firstname#7, lastname#8]\nArguments: org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$2453/0x00007505ed0efbf0@418f2af3, IcebergWrite(table=local.developers, format=PARQUET)\n\n","sparkPlanInfo":{"nodeName":"AppendData","simpleString":"AppendData org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$2453/0x00007505ed0efbf0@418f2af3, IcebergWrite(table=local.developers, format=PARQUET)","children":[{"nodeName":"AdaptiveSparkPlan","simpleString":"AdaptiveSparkPlan isFinalPlan=true","children":[{"nodeName":"ShuffleQueryStage","simpleString":"ShuffleQueryStage 0","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(username#6, 200), REPARTITION_BY_NUM, [plan_id=21]","children":[{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [username#6, firstname#7, lastname#8]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":14,"metricType":"sum"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":12,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":13,"metricType":"nsTiming"},{"name":"records read","accumulatorId":10,"metricType":"sum"},{"name":"local bytes read","accumulatorId":8,"metricType":"size"},{"name":"fetch wait time","accumulatorId":9,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":6,"metricType":"size"},{"name":"local blocks read","accumulatorId":5,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":4,"metricType":"sum"},{"name":"data size","accumulatorId":2,"metricType":"size"},{"name":"number of partitions","accumulatorId":3,"metricType":"sum"},{"name":"remote bytes read to disk","accumulatorId":7,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":11,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}} +{"Event":"SparkListenerJobStart","Job ID":1,"Submission Time":1714515312568,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"processLine at CliDriver.java:336","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"Exchange\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"LocalTableScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"2\",\"name\":\"LocalTableScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"processLine at CliDriver.java:336","Number of Tasks":200,"RDD Info":[{"RDD ID":3,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"4\",\"name\":\"Exchange\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"UNORDERED","Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[1],"Details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[1,2],"Properties":{"spark.dynamicAllocation.shuffleTracking.enabled":"true","spark.sql.warehouse.dir":"file:/home/holden/repos/sparklens/spark-warehouse","spark.executor.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.driver.host":"10.69.200.180","spark.eventLog.enabled":"true","spark.driver.port":"37323","__fetch_continuous_blocks_in_batch_enabled":"true","spark.sql.catalog.local.type":"hadoop","spark.jars":"","spark.sql.catalog.spark_catalog":"org.apache.iceberg.spark.SparkSessionCatalog","spark.kubernetes.authenticate.submission.caCertFile":"/var/snap/microk8s/current/certs/ca.crt","spark.sql.datetime.java8API.enabled":"true","spark.sql.hive.version":"2.3.9","spark.app.name":"fresh-kube","spark.submit.pyFiles":"","spark.app.submitTime":"1714515303940","spark.job.description":"\nINSERT INTO local.developers VALUES(\"krisnova\", \"Kris\", \"Nova\")","spark.kubernetes.container.image":"localhost:32000/local-spark/spark:magic","spark.app.startTime":"1714515304235","spark.executor.id":"driver","spark.sql.catalog.local.warehouse":"/tmp/warehouse","spark.driver.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.sql.catalog.local":"org.apache.iceberg.spark.SparkCatalog","spark.submit.deployMode":"client","spark.master":"k8s://https://127.0.0.1:16443","spark.kubernetes.executor.podNamePrefix":"fresh-kube-bb0c5b8f311360be","spark.dynamicAllocation.enabled":"true","spark.sql.execution.id":"4","spark.sql.catalogImplementation":"hive","spark.kubernetes.authenticate.caCertFile":"/var/snap/microk8s/current/certs/ca.crt","spark.sql.catalog.spark_catalog.type":"hive","spark.app.id":"spark-a8d39974ec7b451a92b303fa953434e4"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"processLine at CliDriver.java:336","Number of Tasks":200,"RDD Info":[{"RDD ID":3,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"4\",\"name\":\"Exchange\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"UNORDERED","Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[1],"Details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1714515312577,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.dynamicAllocation.shuffleTracking.enabled":"true","spark.sql.warehouse.dir":"file:/home/holden/repos/sparklens/spark-warehouse","spark.executor.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.driver.host":"10.69.200.180","spark.eventLog.enabled":"true","spark.driver.port":"37323","__fetch_continuous_blocks_in_batch_enabled":"true","spark.sql.catalog.local.type":"hadoop","spark.jars":"","spark.sql.catalog.spark_catalog":"org.apache.iceberg.spark.SparkSessionCatalog","spark.kubernetes.authenticate.submission.caCertFile":"/var/snap/microk8s/current/certs/ca.crt","spark.sql.datetime.java8API.enabled":"true","spark.sql.hive.version":"2.3.9","spark.app.name":"fresh-kube","spark.submit.pyFiles":"","spark.app.submitTime":"1714515303940","spark.job.description":"\nINSERT INTO local.developers VALUES(\"krisnova\", \"Kris\", \"Nova\")","spark.kubernetes.container.image":"localhost:32000/local-spark/spark:magic","spark.app.startTime":"1714515304235","spark.executor.id":"driver","spark.sql.catalog.local.warehouse":"/tmp/warehouse","spark.driver.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.sql.catalog.local":"org.apache.iceberg.spark.SparkCatalog","spark.submit.deployMode":"client","spark.master":"k8s://https://127.0.0.1:16443","spark.kubernetes.executor.podNamePrefix":"fresh-kube-bb0c5b8f311360be","spark.dynamicAllocation.enabled":"true","spark.sql.execution.id":"4","spark.sql.catalogImplementation":"hive","spark.kubernetes.authenticate.caCertFile":"/var/snap/microk8s/current/certs/ca.crt","spark.sql.catalog.spark_catalog.type":"hive","spark.app.id":"spark-a8d39974ec7b451a92b303fa953434e4"}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":68,"Attempt":0,"Partition ID":68,"Launch Time":1714515312589,"Executor ID":"1","Host":"10.1.246.124","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":0,"Attempt":0,"Partition ID":0,"Launch Time":1714515313279,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":1,"Attempt":0,"Partition ID":1,"Launch Time":1714515313297,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":0,"Attempt":0,"Partition ID":0,"Launch Time":1714515313279,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313298,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2480341,"Value":2480341,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":10,"Value":10,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":8643986,"Value":8643986,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":2513,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":2480341,"Executor Run Time":10,"Executor CPU Time":8643986,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":68,"Attempt":0,"Partition ID":68,"Launch Time":1714515312589,"Executor ID":"1","Host":"10.1.246.124","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313303,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"local blocks read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"local bytes read","Update":"95","Value":"95","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":9,"Name":"fetch wait time","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":10,"Name":"records read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":36,"Value":38,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":31799116,"Value":34279457,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":642,"Value":652,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":536767839,"Value":545411825,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":5281,"Value":7794,"Internal":true,"Count Failed Values":true},{"ID":45,"Name":"internal.metrics.jvmGCTime","Update":62,"Value":62,"Internal":true,"Count Failed Values":true},{"ID":46,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":95,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.output.bytesWritten","Update":932,"Value":932,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.output.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":36,"Executor Deserialize CPU Time":31799116,"Executor Run Time":642,"Executor CPU Time":536767839,"Peak Execution Memory":0,"Result Size":5281,"JVM GC Time":62,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":95,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":932,"Records Written":1},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":2,"Attempt":0,"Partition ID":2,"Launch Time":1714515313308,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":1,"Attempt":0,"Partition ID":1,"Launch Time":1714515313297,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313309,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":40,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2646215,"Value":36925672,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":3,"Value":655,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":2600644,"Value":548012469,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":10307,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":2646215,"Executor Run Time":3,"Executor CPU Time":2600644,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":3,"Attempt":0,"Partition ID":3,"Launch Time":1714515313318,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":2,"Attempt":0,"Partition ID":2,"Launch Time":1714515313308,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313319,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":41,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1742900,"Value":38668572,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":3,"Value":658,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":2554010,"Value":550566479,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":12820,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1742900,"Executor Run Time":3,"Executor CPU Time":2554010,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":4,"Attempt":0,"Partition ID":4,"Launch Time":1714515313326,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":3,"Attempt":0,"Partition ID":3,"Launch Time":1714515313318,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313327,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":42,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1595058,"Value":40263630,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":660,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":2210957,"Value":552777436,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":15333,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1595058,"Executor Run Time":2,"Executor CPU Time":2210957,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":5,"Attempt":0,"Partition ID":5,"Launch Time":1714515313335,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":4,"Attempt":0,"Partition ID":4,"Launch Time":1714515313326,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313336,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":43,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1613806,"Value":41877436,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":3,"Value":663,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":2238315,"Value":555015751,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":17846,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1613806,"Executor Run Time":3,"Executor CPU Time":2238315,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":6,"Attempt":0,"Partition ID":6,"Launch Time":1714515313345,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":5,"Attempt":0,"Partition ID":5,"Launch Time":1714515313335,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313345,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":45,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2151639,"Value":44029075,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":3,"Value":666,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":2500871,"Value":557516622,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":20359,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":2151639,"Executor Run Time":3,"Executor CPU Time":2500871,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":7,"Attempt":0,"Partition ID":7,"Launch Time":1714515313353,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":6,"Attempt":0,"Partition ID":6,"Launch Time":1714515313345,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313354,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":46,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1429587,"Value":45458662,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":3,"Value":669,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":2762278,"Value":560278900,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":22872,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1429587,"Executor Run Time":3,"Executor CPU Time":2762278,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":8,"Attempt":0,"Partition ID":8,"Launch Time":1714515313361,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":7,"Attempt":0,"Partition ID":7,"Launch Time":1714515313353,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313361,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":47,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1564630,"Value":47023292,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":671,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1970790,"Value":562249690,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":25385,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1564630,"Executor Run Time":2,"Executor CPU Time":1970790,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":9,"Attempt":0,"Partition ID":9,"Launch Time":1714515313369,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":8,"Attempt":0,"Partition ID":8,"Launch Time":1714515313361,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313369,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":48,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1292953,"Value":48316245,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":3,"Value":674,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":2523319,"Value":564773009,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":27898,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1292953,"Executor Run Time":3,"Executor CPU Time":2523319,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":10,"Attempt":0,"Partition ID":10,"Launch Time":1714515313376,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":9,"Attempt":0,"Partition ID":9,"Launch Time":1714515313369,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313377,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":49,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1397339,"Value":49713584,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":676,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":2185807,"Value":566958816,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":30411,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1397339,"Executor Run Time":2,"Executor CPU Time":2185807,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":11,"Attempt":0,"Partition ID":11,"Launch Time":1714515313384,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":10,"Attempt":0,"Partition ID":10,"Launch Time":1714515313376,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313385,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":50,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1587228,"Value":51300812,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":678,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1962721,"Value":568921537,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":32924,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1587228,"Executor Run Time":2,"Executor CPU Time":1962721,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":12,"Attempt":0,"Partition ID":12,"Launch Time":1714515313391,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":11,"Attempt":0,"Partition ID":11,"Launch Time":1714515313384,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313392,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":51,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1244026,"Value":52544838,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":680,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1941121,"Value":570862658,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":35437,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1244026,"Executor Run Time":2,"Executor CPU Time":1941121,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":13,"Attempt":0,"Partition ID":13,"Launch Time":1714515313397,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":12,"Attempt":0,"Partition ID":12,"Launch Time":1714515313391,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313397,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":52,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1116162,"Value":53661000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":682,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1704503,"Value":572567161,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":37950,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1116162,"Executor Run Time":2,"Executor CPU Time":1704503,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":14,"Attempt":0,"Partition ID":14,"Launch Time":1714515313404,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":13,"Attempt":0,"Partition ID":13,"Launch Time":1714515313397,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313405,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":53,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1810327,"Value":55471327,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":684,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":2024595,"Value":574591756,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":40463,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1810327,"Executor Run Time":2,"Executor CPU Time":2024595,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":15,"Attempt":0,"Partition ID":15,"Launch Time":1714515313411,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":14,"Attempt":0,"Partition ID":14,"Launch Time":1714515313404,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313412,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":54,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1531492,"Value":57002819,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":686,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1733601,"Value":576325357,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":42976,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1531492,"Executor Run Time":2,"Executor CPU Time":1733601,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":16,"Attempt":0,"Partition ID":16,"Launch Time":1714515313418,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":15,"Attempt":0,"Partition ID":15,"Launch Time":1714515313411,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313418,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":55,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1371639,"Value":58374458,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":688,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1747030,"Value":578072387,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":45489,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1371639,"Executor Run Time":2,"Executor CPU Time":1747030,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":17,"Attempt":0,"Partition ID":17,"Launch Time":1714515313424,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":16,"Attempt":0,"Partition ID":16,"Launch Time":1714515313418,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313424,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":56,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1085754,"Value":59460212,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":690,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1695002,"Value":579767389,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":48002,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1085754,"Executor Run Time":2,"Executor CPU Time":1695002,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":18,"Attempt":0,"Partition ID":18,"Launch Time":1714515313429,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":17,"Attempt":0,"Partition ID":17,"Launch Time":1714515313424,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313430,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":998518,"Value":60458730,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":692,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1560450,"Value":581327839,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":50472,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":998518,"Executor Run Time":2,"Executor CPU Time":1560450,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":19,"Attempt":0,"Partition ID":19,"Launch Time":1714515313436,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":18,"Attempt":0,"Partition ID":18,"Launch Time":1714515313429,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313436,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":57,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1436766,"Value":61895496,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":694,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":2103732,"Value":583431571,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":52985,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1436766,"Executor Run Time":2,"Executor CPU Time":2103732,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":20,"Attempt":0,"Partition ID":20,"Launch Time":1714515313442,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":19,"Attempt":0,"Partition ID":19,"Launch Time":1714515313436,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313443,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":58,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1300243,"Value":63195739,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":696,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1779409,"Value":585210980,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":55498,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1300243,"Executor Run Time":2,"Executor CPU Time":1779409,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":21,"Attempt":0,"Partition ID":21,"Launch Time":1714515313449,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":20,"Attempt":0,"Partition ID":20,"Launch Time":1714515313442,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313449,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":59,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1158770,"Value":64354509,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":698,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1819107,"Value":587030087,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":58011,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1158770,"Executor Run Time":2,"Executor CPU Time":1819107,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":22,"Attempt":0,"Partition ID":22,"Launch Time":1714515313456,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":21,"Attempt":0,"Partition ID":21,"Launch Time":1714515313449,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313456,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1447956,"Value":65802465,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":700,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1946939,"Value":588977026,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":60524,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1447956,"Executor Run Time":2,"Executor CPU Time":1946939,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":23,"Attempt":0,"Partition ID":23,"Launch Time":1714515313462,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":22,"Attempt":0,"Partition ID":22,"Launch Time":1714515313456,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313462,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":61,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1094783,"Value":66897248,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":702,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1692744,"Value":590669770,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":63037,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1094783,"Executor Run Time":2,"Executor CPU Time":1692744,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":24,"Attempt":0,"Partition ID":24,"Launch Time":1714515313469,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":23,"Attempt":0,"Partition ID":23,"Launch Time":1714515313462,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313470,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":62,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1151931,"Value":68049179,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":3,"Value":705,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":2500152,"Value":593169922,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":65550,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1151931,"Executor Run Time":3,"Executor CPU Time":2500152,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":25,"Attempt":0,"Partition ID":25,"Launch Time":1714515313477,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":24,"Attempt":0,"Partition ID":24,"Launch Time":1714515313469,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313477,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1386938,"Value":69436117,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":707,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":2140260,"Value":595310182,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":68063,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1386938,"Executor Run Time":2,"Executor CPU Time":2140260,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":26,"Attempt":0,"Partition ID":26,"Launch Time":1714515313483,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":25,"Attempt":0,"Partition ID":25,"Launch Time":1714515313477,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313484,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":64,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1131833,"Value":70567950,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":709,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1715700,"Value":597025882,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":70576,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1131833,"Executor Run Time":2,"Executor CPU Time":1715700,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":27,"Attempt":0,"Partition ID":27,"Launch Time":1714515313489,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":26,"Attempt":0,"Partition ID":26,"Launch Time":1714515313483,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313490,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":65,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1122452,"Value":71690402,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":711,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1827886,"Value":598853768,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":73089,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1122452,"Executor Run Time":2,"Executor CPU Time":1827886,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":28,"Attempt":0,"Partition ID":28,"Launch Time":1714515313495,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":27,"Attempt":0,"Partition ID":27,"Launch Time":1714515313489,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313495,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":66,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1158260,"Value":72848662,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":713,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1767310,"Value":600621078,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":75602,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1158260,"Executor Run Time":2,"Executor CPU Time":1767310,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":29,"Attempt":0,"Partition ID":29,"Launch Time":1714515313501,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":28,"Attempt":0,"Partition ID":28,"Launch Time":1714515313495,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313501,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":949430,"Value":73798092,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":715,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1673784,"Value":602294862,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":78072,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":949430,"Executor Run Time":2,"Executor CPU Time":1673784,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":32,"Index":30,"Attempt":0,"Partition ID":30,"Launch Time":1714515313506,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":31,"Index":29,"Attempt":0,"Partition ID":29,"Launch Time":1714515313501,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313507,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":998919,"Value":74797011,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":717,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1641674,"Value":603936536,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":80542,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":998919,"Executor Run Time":2,"Executor CPU Time":1641674,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":33,"Index":31,"Attempt":0,"Partition ID":31,"Launch Time":1714515313512,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":32,"Index":30,"Attempt":0,"Partition ID":30,"Launch Time":1714515313506,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313513,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":67,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1049126,"Value":75846137,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":719,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1535772,"Value":605472308,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":83055,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1049126,"Executor Run Time":2,"Executor CPU Time":1535772,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":34,"Index":32,"Attempt":0,"Partition ID":32,"Launch Time":1714515313521,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":33,"Index":31,"Attempt":0,"Partition ID":31,"Launch Time":1714515313512,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313521,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":68,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1668455,"Value":77514592,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":3,"Value":722,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":3040394,"Value":608512702,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":85568,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1668455,"Executor Run Time":3,"Executor CPU Time":3040394,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":35,"Index":33,"Attempt":0,"Partition ID":33,"Launch Time":1714515313528,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":34,"Index":32,"Attempt":0,"Partition ID":32,"Launch Time":1714515313521,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313529,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":69,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1684893,"Value":79199485,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":724,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":2216307,"Value":610729009,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":88081,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1684893,"Executor Run Time":2,"Executor CPU Time":2216307,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":36,"Index":34,"Attempt":0,"Partition ID":34,"Launch Time":1714515313535,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":35,"Index":33,"Attempt":0,"Partition ID":33,"Launch Time":1714515313528,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313536,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":70,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1296893,"Value":80496378,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":3,"Value":727,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":2401307,"Value":613130316,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":90594,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1296893,"Executor Run Time":3,"Executor CPU Time":2401307,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":37,"Index":35,"Attempt":0,"Partition ID":35,"Launch Time":1714515313542,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":36,"Index":34,"Attempt":0,"Partition ID":34,"Launch Time":1714515313535,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313542,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":71,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1007647,"Value":81504025,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":729,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1937861,"Value":615068177,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":93107,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1007647,"Executor Run Time":2,"Executor CPU Time":1937861,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":38,"Index":36,"Attempt":0,"Partition ID":36,"Launch Time":1714515313548,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":37,"Index":35,"Attempt":0,"Partition ID":35,"Launch Time":1714515313542,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313548,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":72,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1252826,"Value":82756851,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":731,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1694513,"Value":616762690,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":95620,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1252826,"Executor Run Time":2,"Executor CPU Time":1694513,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":39,"Index":37,"Attempt":0,"Partition ID":37,"Launch Time":1714515313554,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":38,"Index":36,"Attempt":0,"Partition ID":36,"Launch Time":1714515313548,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313554,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":73,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1301552,"Value":84058403,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":733,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1785779,"Value":618548469,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":98133,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1301552,"Executor Run Time":2,"Executor CPU Time":1785779,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":40,"Index":38,"Attempt":0,"Partition ID":38,"Launch Time":1714515313559,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":39,"Index":37,"Attempt":0,"Partition ID":37,"Launch Time":1714515313554,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313559,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":999179,"Value":85057582,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":734,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1466715,"Value":620015184,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":100603,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":999179,"Executor Run Time":1,"Executor CPU Time":1466715,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":41,"Index":39,"Attempt":0,"Partition ID":39,"Launch Time":1714515313564,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":40,"Index":38,"Attempt":0,"Partition ID":38,"Launch Time":1714515313559,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313565,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":74,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1063275,"Value":86120857,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":736,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1591649,"Value":621606833,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":103116,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1063275,"Executor Run Time":2,"Executor CPU Time":1591649,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":42,"Index":40,"Attempt":0,"Partition ID":40,"Launch Time":1714515313569,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":41,"Index":39,"Attempt":0,"Partition ID":39,"Launch Time":1714515313564,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313570,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":75,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1171470,"Value":87292327,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":738,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1549540,"Value":623156373,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":105629,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1171470,"Executor Run Time":2,"Executor CPU Time":1549540,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":43,"Index":41,"Attempt":0,"Partition ID":41,"Launch Time":1714515313575,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":42,"Index":40,"Attempt":0,"Partition ID":40,"Launch Time":1714515313569,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313576,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":76,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1043426,"Value":88335753,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":740,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1621668,"Value":624778041,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":108142,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1043426,"Executor Run Time":2,"Executor CPU Time":1621668,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":44,"Index":42,"Attempt":0,"Partition ID":42,"Launch Time":1714515313580,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":43,"Index":41,"Attempt":0,"Partition ID":41,"Launch Time":1714515313575,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313580,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":828357,"Value":89164110,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":741,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1273624,"Value":626051665,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":110612,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":828357,"Executor Run Time":1,"Executor CPU Time":1273624,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":45,"Index":43,"Attempt":0,"Partition ID":43,"Launch Time":1714515313585,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":44,"Index":42,"Attempt":0,"Partition ID":42,"Launch Time":1714515313580,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313586,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":766821,"Value":89930931,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":743,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1522882,"Value":627574547,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":113082,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":766821,"Executor Run Time":2,"Executor CPU Time":1522882,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":46,"Index":44,"Attempt":0,"Partition ID":44,"Launch Time":1714515313590,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":45,"Index":43,"Attempt":0,"Partition ID":43,"Launch Time":1714515313585,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313591,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":77,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1001949,"Value":90932880,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":744,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1392928,"Value":628967475,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":115595,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1001949,"Executor Run Time":1,"Executor CPU Time":1392928,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":47,"Index":45,"Attempt":0,"Partition ID":45,"Launch Time":1714515313595,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":46,"Index":44,"Attempt":0,"Partition ID":44,"Launch Time":1714515313590,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313595,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":862105,"Value":91794985,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":745,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1273774,"Value":630241249,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":118065,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":862105,"Executor Run Time":1,"Executor CPU Time":1273774,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":48,"Index":46,"Attempt":0,"Partition ID":46,"Launch Time":1714515313600,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":47,"Index":45,"Attempt":0,"Partition ID":45,"Launch Time":1714515313595,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313600,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":751301,"Value":92546286,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":746,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1150243,"Value":631391492,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":120535,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":751301,"Executor Run Time":1,"Executor CPU Time":1150243,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":49,"Index":47,"Attempt":0,"Partition ID":47,"Launch Time":1714515313606,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":48,"Index":46,"Attempt":0,"Partition ID":46,"Launch Time":1714515313600,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313606,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":922923,"Value":93469209,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":748,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1552650,"Value":632944142,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":123005,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":922923,"Executor Run Time":2,"Executor CPU Time":1552650,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":50,"Index":48,"Attempt":0,"Partition ID":48,"Launch Time":1714515313612,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":49,"Index":47,"Attempt":0,"Partition ID":47,"Launch Time":1714515313606,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313612,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":78,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1050646,"Value":94519855,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":750,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1558759,"Value":634502901,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":125518,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1050646,"Executor Run Time":2,"Executor CPU Time":1558759,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":51,"Index":49,"Attempt":0,"Partition ID":49,"Launch Time":1714515313617,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":50,"Index":48,"Attempt":0,"Partition ID":48,"Launch Time":1714515313612,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313618,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":921962,"Value":95441817,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":752,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1441327,"Value":635944228,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":127988,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":921962,"Executor Run Time":2,"Executor CPU Time":1441327,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":52,"Index":50,"Attempt":0,"Partition ID":50,"Launch Time":1714515313623,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":51,"Index":49,"Attempt":0,"Partition ID":49,"Launch Time":1714515313617,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313623,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":946181,"Value":96387998,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":753,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1373149,"Value":637317377,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":130458,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":946181,"Executor Run Time":1,"Executor CPU Time":1373149,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":53,"Index":51,"Attempt":0,"Partition ID":51,"Launch Time":1714515313627,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":52,"Index":50,"Attempt":0,"Partition ID":50,"Launch Time":1714515313623,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313628,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":762201,"Value":97150199,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":754,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1134182,"Value":638451559,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":132928,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":762201,"Executor Run Time":1,"Executor CPU Time":1134182,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":54,"Index":52,"Attempt":0,"Partition ID":52,"Launch Time":1714515313632,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":53,"Index":51,"Attempt":0,"Partition ID":51,"Launch Time":1714515313627,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313633,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":818207,"Value":97968406,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":755,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1040906,"Value":639492465,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":135398,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":818207,"Executor Run Time":1,"Executor CPU Time":1040906,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":55,"Index":53,"Attempt":0,"Partition ID":53,"Launch Time":1714515313638,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":54,"Index":52,"Attempt":0,"Partition ID":52,"Launch Time":1714515313632,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313639,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":962221,"Value":98930627,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":756,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1413657,"Value":640906122,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":137868,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":962221,"Executor Run Time":1,"Executor CPU Time":1413657,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":56,"Index":54,"Attempt":0,"Partition ID":54,"Launch Time":1714515313644,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":55,"Index":53,"Attempt":0,"Partition ID":53,"Launch Time":1714515313638,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313645,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":962671,"Value":99893298,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":758,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1576268,"Value":642482390,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":140338,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":962671,"Executor Run Time":2,"Executor CPU Time":1576268,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":57,"Index":55,"Attempt":0,"Partition ID":55,"Launch Time":1714515313651,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":56,"Index":54,"Attempt":0,"Partition ID":54,"Launch Time":1714515313644,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313651,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":79,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1027747,"Value":100921045,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":760,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1778268,"Value":644260658,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":142851,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1027747,"Executor Run Time":2,"Executor CPU Time":1778268,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":58,"Index":56,"Attempt":0,"Partition ID":56,"Launch Time":1714515313657,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":57,"Index":55,"Attempt":0,"Partition ID":55,"Launch Time":1714515313651,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313657,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":80,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1100694,"Value":102021739,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":762,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1565109,"Value":645825767,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":145364,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1100694,"Executor Run Time":2,"Executor CPU Time":1565109,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":59,"Index":57,"Attempt":0,"Partition ID":57,"Launch Time":1714515313662,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":58,"Index":56,"Attempt":0,"Partition ID":56,"Launch Time":1714515313657,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313662,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":883825,"Value":102905564,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":763,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1360899,"Value":647186666,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":147834,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":883825,"Executor Run Time":1,"Executor CPU Time":1360899,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":60,"Index":58,"Attempt":0,"Partition ID":58,"Launch Time":1714515313667,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":59,"Index":57,"Attempt":0,"Partition ID":57,"Launch Time":1714515313662,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313668,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":801588,"Value":103707152,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":764,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1220607,"Value":648407273,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":150304,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":801588,"Executor Run Time":1,"Executor CPU Time":1220607,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":61,"Index":59,"Attempt":0,"Partition ID":59,"Launch Time":1714515313673,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":60,"Index":58,"Attempt":0,"Partition ID":58,"Launch Time":1714515313667,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313673,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":954430,"Value":104661582,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":765,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1473504,"Value":649880777,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":152774,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":954430,"Executor Run Time":1,"Executor CPU Time":1473504,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":62,"Index":60,"Attempt":0,"Partition ID":60,"Launch Time":1714515313679,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":61,"Index":59,"Attempt":0,"Partition ID":59,"Launch Time":1714515313673,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313680,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":81,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1185249,"Value":105846831,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":767,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1613267,"Value":651494044,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":155287,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1185249,"Executor Run Time":2,"Executor CPU Time":1613267,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":63,"Index":61,"Attempt":0,"Partition ID":61,"Launch Time":1714515313688,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":62,"Index":60,"Attempt":0,"Partition ID":60,"Launch Time":1714515313679,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313688,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":82,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1606478,"Value":107453309,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":769,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1946289,"Value":653440333,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":157800,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1606478,"Executor Run Time":2,"Executor CPU Time":1946289,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":64,"Index":62,"Attempt":0,"Partition ID":62,"Launch Time":1714515313694,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":63,"Index":61,"Attempt":0,"Partition ID":61,"Launch Time":1714515313688,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313694,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":83,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1036217,"Value":108489526,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":771,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1431745,"Value":654872078,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":160313,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1036217,"Executor Run Time":2,"Executor CPU Time":1431745,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":65,"Index":63,"Attempt":0,"Partition ID":63,"Launch Time":1714515313702,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":64,"Index":62,"Attempt":0,"Partition ID":62,"Launch Time":1714515313694,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313702,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1132241,"Value":109621767,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":3,"Value":774,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":2167649,"Value":657039727,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":162826,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1132241,"Executor Run Time":3,"Executor CPU Time":2167649,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":66,"Index":64,"Attempt":0,"Partition ID":64,"Launch Time":1714515313709,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":65,"Index":63,"Attempt":0,"Partition ID":63,"Launch Time":1714515313702,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313709,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":85,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1232917,"Value":110854684,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":776,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1693113,"Value":658732840,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":165339,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1232917,"Executor Run Time":2,"Executor CPU Time":1693113,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":67,"Index":65,"Attempt":0,"Partition ID":65,"Launch Time":1714515313714,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":66,"Index":64,"Attempt":0,"Partition ID":64,"Launch Time":1714515313709,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313715,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":978600,"Value":111833284,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":777,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1467954,"Value":660200794,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":167809,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":978600,"Executor Run Time":1,"Executor CPU Time":1467954,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":68,"Index":66,"Attempt":0,"Partition ID":66,"Launch Time":1714515313721,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":67,"Index":65,"Attempt":0,"Partition ID":65,"Launch Time":1714515313714,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313722,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":86,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1590748,"Value":113424032,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":779,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1858564,"Value":662059358,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":170322,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1590748,"Executor Run Time":2,"Executor CPU Time":1858564,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":69,"Index":67,"Attempt":0,"Partition ID":67,"Launch Time":1714515313728,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":68,"Index":66,"Attempt":0,"Partition ID":66,"Launch Time":1714515313721,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313728,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":87,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1136951,"Value":114560983,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":781,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1572620,"Value":663631978,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":172835,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1136951,"Executor Run Time":2,"Executor CPU Time":1572620,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":70,"Index":69,"Attempt":0,"Partition ID":69,"Launch Time":1714515313733,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":69,"Index":67,"Attempt":0,"Partition ID":67,"Launch Time":1714515313728,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313734,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":894904,"Value":115455887,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":782,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1375029,"Value":665007007,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":175305,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":894904,"Executor Run Time":1,"Executor CPU Time":1375029,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":71,"Index":70,"Attempt":0,"Partition ID":70,"Launch Time":1714515313738,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":70,"Index":69,"Attempt":0,"Partition ID":69,"Launch Time":1714515313733,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313739,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":929202,"Value":116385089,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":783,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1395158,"Value":666402165,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":177775,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":929202,"Executor Run Time":1,"Executor CPU Time":1395158,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":72,"Index":71,"Attempt":0,"Partition ID":71,"Launch Time":1714515313744,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":71,"Index":70,"Attempt":0,"Partition ID":70,"Launch Time":1714515313738,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313744,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":88,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1127272,"Value":117512361,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":785,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1501642,"Value":667903807,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":180288,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1127272,"Executor Run Time":2,"Executor CPU Time":1501642,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":73,"Index":72,"Attempt":0,"Partition ID":72,"Launch Time":1714515313749,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":72,"Index":71,"Attempt":0,"Partition ID":71,"Launch Time":1714515313744,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313750,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":915493,"Value":118427854,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":786,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1367119,"Value":669270926,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":182758,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":915493,"Executor Run Time":1,"Executor CPU Time":1367119,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":74,"Index":73,"Attempt":0,"Partition ID":73,"Launch Time":1714515313755,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":73,"Index":72,"Attempt":0,"Partition ID":72,"Launch Time":1714515313749,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313756,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":89,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1220338,"Value":119648192,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":788,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1452265,"Value":670723191,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":185271,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1220338,"Executor Run Time":2,"Executor CPU Time":1452265,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":75,"Index":74,"Attempt":0,"Partition ID":74,"Launch Time":1714515313760,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":74,"Index":73,"Attempt":0,"Partition ID":73,"Launch Time":1714515313755,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313760,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":852996,"Value":120501188,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":789,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1222297,"Value":671945488,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":187741,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":852996,"Executor Run Time":1,"Executor CPU Time":1222297,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":76,"Index":75,"Attempt":0,"Partition ID":75,"Launch Time":1714515313764,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":75,"Index":74,"Attempt":0,"Partition ID":74,"Launch Time":1714515313760,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313765,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":90,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1027427,"Value":121528615,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":790,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1285664,"Value":673231152,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":190254,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1027427,"Executor Run Time":1,"Executor CPU Time":1285664,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":77,"Index":76,"Attempt":0,"Partition ID":76,"Launch Time":1714515313771,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":76,"Index":75,"Attempt":0,"Partition ID":75,"Launch Time":1714515313764,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313771,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":91,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1208818,"Value":122737433,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":792,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1643665,"Value":674874817,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":192767,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1208818,"Executor Run Time":2,"Executor CPU Time":1643665,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":78,"Index":77,"Attempt":0,"Partition ID":77,"Launch Time":1714515313776,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":77,"Index":76,"Attempt":0,"Partition ID":76,"Launch Time":1714515313771,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313777,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":935330,"Value":123672763,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":794,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1467416,"Value":676342233,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":195237,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":935330,"Executor Run Time":2,"Executor CPU Time":1467416,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":79,"Index":78,"Attempt":0,"Partition ID":78,"Launch Time":1714515313782,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":78,"Index":77,"Attempt":0,"Partition ID":77,"Launch Time":1714515313776,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313782,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":92,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1044255,"Value":124717018,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":796,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1374441,"Value":677716674,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":197750,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1044255,"Executor Run Time":2,"Executor CPU Time":1374441,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":80,"Index":79,"Attempt":0,"Partition ID":79,"Launch Time":1714515313787,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":79,"Index":78,"Attempt":0,"Partition ID":78,"Launch Time":1714515313782,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313788,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":967390,"Value":125684408,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":797,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1361070,"Value":679077744,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":200220,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":967390,"Executor Run Time":1,"Executor CPU Time":1361070,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":81,"Index":80,"Attempt":0,"Partition ID":80,"Launch Time":1714515313792,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":80,"Index":79,"Attempt":0,"Partition ID":79,"Launch Time":1714515313787,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313792,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":841137,"Value":126525545,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":798,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1203838,"Value":680281582,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":202690,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":841137,"Executor Run Time":1,"Executor CPU Time":1203838,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":82,"Index":81,"Attempt":0,"Partition ID":81,"Launch Time":1714515313795,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":81,"Index":80,"Attempt":0,"Partition ID":80,"Launch Time":1714515313792,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313796,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":701694,"Value":127227239,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":799,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1062775,"Value":681344357,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":205160,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":701694,"Executor Run Time":1,"Executor CPU Time":1062775,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":83,"Index":82,"Attempt":0,"Partition ID":82,"Launch Time":1714515313800,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":82,"Index":81,"Attempt":0,"Partition ID":81,"Launch Time":1714515313795,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313800,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":698514,"Value":127925753,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":800,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1150951,"Value":682495308,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":207630,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":698514,"Executor Run Time":1,"Executor CPU Time":1150951,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":84,"Index":83,"Attempt":0,"Partition ID":83,"Launch Time":1714515313805,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":83,"Index":82,"Attempt":0,"Partition ID":82,"Launch Time":1714515313800,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313806,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":811468,"Value":128737221,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":801,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1102214,"Value":683597522,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":210100,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":811468,"Executor Run Time":1,"Executor CPU Time":1102214,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":85,"Index":84,"Attempt":0,"Partition ID":84,"Launch Time":1714515313810,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":84,"Index":83,"Attempt":0,"Partition ID":83,"Launch Time":1714515313805,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313810,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":824328,"Value":129561549,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":802,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1159050,"Value":684756572,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":212570,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":824328,"Executor Run Time":1,"Executor CPU Time":1159050,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":86,"Index":85,"Attempt":0,"Partition ID":85,"Launch Time":1714515313813,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":85,"Index":84,"Attempt":0,"Partition ID":84,"Launch Time":1714515313810,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313814,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":701624,"Value":130263173,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":803,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1032487,"Value":685789059,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":215040,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":701624,"Executor Run Time":1,"Executor CPU Time":1032487,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":87,"Index":86,"Attempt":0,"Partition ID":86,"Launch Time":1714515313818,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":86,"Index":85,"Attempt":0,"Partition ID":85,"Launch Time":1714515313813,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313818,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":730142,"Value":130993315,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":804,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1304443,"Value":687093502,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":217510,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":730142,"Executor Run Time":1,"Executor CPU Time":1304443,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":88,"Index":87,"Attempt":0,"Partition ID":87,"Launch Time":1714515313822,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":87,"Index":86,"Attempt":0,"Partition ID":86,"Launch Time":1714515313818,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313822,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":892484,"Value":131885799,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":805,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1105013,"Value":688198515,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":219980,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":892484,"Executor Run Time":1,"Executor CPU Time":1105013,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":89,"Index":88,"Attempt":0,"Partition ID":88,"Launch Time":1714515313825,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":88,"Index":87,"Attempt":0,"Partition ID":87,"Launch Time":1714515313822,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313826,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":635518,"Value":132521317,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":806,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":905512,"Value":689104027,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":222450,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":635518,"Executor Run Time":1,"Executor CPU Time":905512,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":90,"Index":89,"Attempt":0,"Partition ID":89,"Launch Time":1714515313829,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":89,"Index":88,"Attempt":0,"Partition ID":88,"Launch Time":1714515313825,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313829,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":632378,"Value":133153695,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":807,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":974559,"Value":690078586,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":224920,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":632378,"Executor Run Time":1,"Executor CPU Time":974559,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":91,"Index":90,"Attempt":0,"Partition ID":90,"Launch Time":1714515313832,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":90,"Index":89,"Attempt":0,"Partition ID":89,"Launch Time":1714515313829,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313832,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":579940,"Value":133733635,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":808,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":868165,"Value":690946751,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":227390,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":579940,"Executor Run Time":1,"Executor CPU Time":868165,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":92,"Index":91,"Attempt":0,"Partition ID":91,"Launch Time":1714515313835,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":91,"Index":90,"Attempt":0,"Partition ID":90,"Launch Time":1714515313832,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313835,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":808429,"Value":134542064,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":809,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":963010,"Value":691909761,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":229860,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":808429,"Executor Run Time":1,"Executor CPU Time":963010,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":93,"Index":92,"Attempt":0,"Partition ID":92,"Launch Time":1714515313838,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":92,"Index":91,"Attempt":0,"Partition ID":91,"Launch Time":1714515313835,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313839,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":564141,"Value":135106205,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":810,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":868725,"Value":692778486,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":232330,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":564141,"Executor Run Time":1,"Executor CPU Time":868725,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":94,"Index":93,"Attempt":0,"Partition ID":93,"Launch Time":1714515313842,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":93,"Index":92,"Attempt":0,"Partition ID":92,"Launch Time":1714515313838,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313842,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":655447,"Value":135761652,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":811,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":987698,"Value":693766184,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":234800,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":655447,"Executor Run Time":1,"Executor CPU Time":987698,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":95,"Index":94,"Attempt":0,"Partition ID":94,"Launch Time":1714515313845,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":94,"Index":93,"Attempt":0,"Partition ID":93,"Launch Time":1714515313842,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313846,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":660395,"Value":136422047,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":812,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":934492,"Value":694700676,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":237270,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":660395,"Executor Run Time":1,"Executor CPU Time":934492,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":96,"Index":95,"Attempt":0,"Partition ID":95,"Launch Time":1714515313849,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":95,"Index":94,"Attempt":0,"Partition ID":94,"Launch Time":1714515313845,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313849,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":624018,"Value":137046065,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":813,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":949131,"Value":695649807,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":239740,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":624018,"Executor Run Time":1,"Executor CPU Time":949131,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":97,"Index":96,"Attempt":0,"Partition ID":96,"Launch Time":1714515313853,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":96,"Index":95,"Attempt":0,"Partition ID":95,"Launch Time":1714515313849,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313854,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":842087,"Value":137888152,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":814,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1241576,"Value":696891383,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":242210,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":842087,"Executor Run Time":1,"Executor CPU Time":1241576,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":98,"Index":97,"Attempt":0,"Partition ID":97,"Launch Time":1714515313857,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":97,"Index":96,"Attempt":0,"Partition ID":96,"Launch Time":1714515313853,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313857,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":694224,"Value":138582376,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":815,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1006999,"Value":697898382,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":244680,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":694224,"Executor Run Time":1,"Executor CPU Time":1006999,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":99,"Index":98,"Attempt":0,"Partition ID":98,"Launch Time":1714515313860,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":98,"Index":97,"Attempt":0,"Partition ID":97,"Launch Time":1714515313857,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313860,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":570381,"Value":139152757,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":816,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":855156,"Value":698753538,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":247150,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":570381,"Executor Run Time":1,"Executor CPU Time":855156,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":100,"Index":99,"Attempt":0,"Partition ID":99,"Launch Time":1714515313863,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":99,"Index":98,"Attempt":0,"Partition ID":98,"Launch Time":1714515313860,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313863,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":564160,"Value":139716917,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":817,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":800059,"Value":699553597,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":249620,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":564160,"Executor Run Time":1,"Executor CPU Time":800059,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":101,"Index":100,"Attempt":0,"Partition ID":100,"Launch Time":1714515313866,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":100,"Index":99,"Attempt":0,"Partition ID":99,"Launch Time":1714515313863,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313867,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":548592,"Value":140265509,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":818,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":797878,"Value":700351475,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":252090,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":548592,"Executor Run Time":1,"Executor CPU Time":797878,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":102,"Index":101,"Attempt":0,"Partition ID":101,"Launch Time":1714515313870,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":101,"Index":100,"Attempt":0,"Partition ID":100,"Launch Time":1714515313866,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313870,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":616028,"Value":140881537,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":819,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":885135,"Value":701236610,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":254560,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":616028,"Executor Run Time":1,"Executor CPU Time":885135,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":103,"Index":102,"Attempt":0,"Partition ID":102,"Launch Time":1714515313873,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":102,"Index":101,"Attempt":0,"Partition ID":101,"Launch Time":1714515313870,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313873,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":521973,"Value":141403510,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":820,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":865056,"Value":702101666,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":257030,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":521973,"Executor Run Time":1,"Executor CPU Time":865056,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":104,"Index":103,"Attempt":0,"Partition ID":103,"Launch Time":1714515313876,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":103,"Index":102,"Attempt":0,"Partition ID":102,"Launch Time":1714515313873,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313876,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":594200,"Value":141997710,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":821,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":901433,"Value":703003099,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":259500,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":594200,"Executor Run Time":1,"Executor CPU Time":901433,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":105,"Index":104,"Attempt":0,"Partition ID":104,"Launch Time":1714515313879,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":104,"Index":103,"Attempt":0,"Partition ID":103,"Launch Time":1714515313876,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313880,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":583220,"Value":142580930,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":822,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":850376,"Value":703853475,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":261970,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":583220,"Executor Run Time":1,"Executor CPU Time":850376,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":106,"Index":105,"Attempt":0,"Partition ID":105,"Launch Time":1714515313883,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":105,"Index":104,"Attempt":0,"Partition ID":104,"Launch Time":1714515313879,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313883,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":586269,"Value":143167199,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":823,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":940243,"Value":704793718,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":264440,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":586269,"Executor Run Time":1,"Executor CPU Time":940243,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":107,"Index":106,"Attempt":0,"Partition ID":106,"Launch Time":1714515313887,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":106,"Index":105,"Attempt":0,"Partition ID":105,"Launch Time":1714515313883,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313887,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":672226,"Value":143839425,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":824,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1004158,"Value":705797876,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":266910,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":672226,"Executor Run Time":1,"Executor CPU Time":1004158,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":108,"Index":107,"Attempt":0,"Partition ID":107,"Launch Time":1714515313890,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":107,"Index":106,"Attempt":0,"Partition ID":106,"Launch Time":1714515313887,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313890,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":588710,"Value":144428135,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":825,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":898064,"Value":706695940,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":269380,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":588710,"Executor Run Time":1,"Executor CPU Time":898064,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":109,"Index":108,"Attempt":0,"Partition ID":108,"Launch Time":1714515313893,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":108,"Index":107,"Attempt":0,"Partition ID":107,"Launch Time":1714515313890,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313893,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":581441,"Value":145009576,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":826,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":822118,"Value":707518058,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":271850,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":581441,"Executor Run Time":1,"Executor CPU Time":822118,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":110,"Index":109,"Attempt":0,"Partition ID":109,"Launch Time":1714515313896,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":109,"Index":108,"Attempt":0,"Partition ID":108,"Launch Time":1714515313893,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313897,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":605879,"Value":145615455,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":827,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":804868,"Value":708322926,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":274320,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":605879,"Executor Run Time":1,"Executor CPU Time":804868,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":111,"Index":110,"Attempt":0,"Partition ID":110,"Launch Time":1714515313900,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":110,"Index":109,"Attempt":0,"Partition ID":109,"Launch Time":1714515313896,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313900,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":535303,"Value":146150758,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":828,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":797828,"Value":709120754,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":276790,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":535303,"Executor Run Time":1,"Executor CPU Time":797828,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":112,"Index":111,"Attempt":0,"Partition ID":111,"Launch Time":1714515313904,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":111,"Index":110,"Attempt":0,"Partition ID":110,"Launch Time":1714515313900,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313904,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":803809,"Value":146954567,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":829,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1067325,"Value":710188079,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":279260,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":803809,"Executor Run Time":1,"Executor CPU Time":1067325,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":113,"Index":112,"Attempt":0,"Partition ID":112,"Launch Time":1714515313907,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":112,"Index":111,"Attempt":0,"Partition ID":111,"Launch Time":1714515313904,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313907,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":594629,"Value":147549196,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":830,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":911113,"Value":711099192,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":281730,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":594629,"Executor Run Time":1,"Executor CPU Time":911113,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":114,"Index":113,"Attempt":0,"Partition ID":113,"Launch Time":1714515313910,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":113,"Index":112,"Attempt":0,"Partition ID":112,"Launch Time":1714515313907,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313911,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":606269,"Value":148155465,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":831,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":794409,"Value":711893601,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":284200,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":606269,"Executor Run Time":1,"Executor CPU Time":794409,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":115,"Index":114,"Attempt":0,"Partition ID":114,"Launch Time":1714515313913,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":114,"Index":113,"Attempt":0,"Partition ID":113,"Launch Time":1714515313910,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313914,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":522684,"Value":148678149,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":832,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":784549,"Value":712678150,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":286670,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":522684,"Executor Run Time":1,"Executor CPU Time":784549,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":116,"Index":115,"Attempt":0,"Partition ID":115,"Launch Time":1714515313917,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":115,"Index":114,"Attempt":0,"Partition ID":114,"Launch Time":1714515313913,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313917,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":577110,"Value":149255259,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":833,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":880435,"Value":713558585,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":289140,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":577110,"Executor Run Time":1,"Executor CPU Time":880435,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":117,"Index":116,"Attempt":0,"Partition ID":116,"Launch Time":1714515313921,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":116,"Index":115,"Attempt":0,"Partition ID":115,"Launch Time":1714515313917,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313922,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":650996,"Value":149906255,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":834,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":997289,"Value":714555874,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":291610,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":650996,"Executor Run Time":1,"Executor CPU Time":997289,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":118,"Index":117,"Attempt":0,"Partition ID":117,"Launch Time":1714515313925,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":117,"Index":116,"Attempt":0,"Partition ID":116,"Launch Time":1714515313921,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313925,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":645518,"Value":150551773,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":835,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":929241,"Value":715485115,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":294080,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":645518,"Executor Run Time":1,"Executor CPU Time":929241,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":119,"Index":118,"Attempt":0,"Partition ID":118,"Launch Time":1714515313928,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":118,"Index":117,"Attempt":0,"Partition ID":117,"Launch Time":1714515313925,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313928,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":620888,"Value":151172661,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":836,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":923372,"Value":716408487,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":296550,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":620888,"Executor Run Time":1,"Executor CPU Time":923372,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":120,"Index":119,"Attempt":0,"Partition ID":119,"Launch Time":1714515313932,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":119,"Index":118,"Attempt":0,"Partition ID":118,"Launch Time":1714515313928,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313932,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":562520,"Value":151735181,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":837,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":828358,"Value":717236845,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":299020,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":562520,"Executor Run Time":1,"Executor CPU Time":828358,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":121,"Index":120,"Attempt":0,"Partition ID":120,"Launch Time":1714515313936,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":120,"Index":119,"Attempt":0,"Partition ID":119,"Launch Time":1714515313932,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313937,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":983479,"Value":152718660,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":838,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1385279,"Value":718622124,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":301490,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":983479,"Executor Run Time":1,"Executor CPU Time":1385279,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":122,"Index":121,"Attempt":0,"Partition ID":121,"Launch Time":1714515313940,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":121,"Index":120,"Attempt":0,"Partition ID":120,"Launch Time":1714515313936,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313940,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":653176,"Value":153371836,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":839,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":959811,"Value":719581935,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":303960,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":653176,"Executor Run Time":1,"Executor CPU Time":959811,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":123,"Index":122,"Attempt":0,"Partition ID":122,"Launch Time":1714515313943,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":122,"Index":121,"Attempt":0,"Partition ID":121,"Launch Time":1714515313940,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313943,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":626657,"Value":153998493,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":840,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":951382,"Value":720533317,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":306430,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":626657,"Executor Run Time":1,"Executor CPU Time":951382,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":124,"Index":123,"Attempt":0,"Partition ID":123,"Launch Time":1714515313946,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":123,"Index":122,"Attempt":0,"Partition ID":122,"Launch Time":1714515313943,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313947,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":600670,"Value":154599163,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":841,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":876153,"Value":721409470,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":308900,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":600670,"Executor Run Time":1,"Executor CPU Time":876153,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":125,"Index":124,"Attempt":0,"Partition ID":124,"Launch Time":1714515313950,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":124,"Index":123,"Attempt":0,"Partition ID":123,"Launch Time":1714515313946,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313951,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":667896,"Value":155267059,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":842,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":963420,"Value":722372890,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":311370,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":667896,"Executor Run Time":1,"Executor CPU Time":963420,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":126,"Index":125,"Attempt":0,"Partition ID":125,"Launch Time":1714515313954,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":125,"Index":124,"Attempt":0,"Partition ID":124,"Launch Time":1714515313950,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313955,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":726263,"Value":155993322,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":843,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1041036,"Value":723413926,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":313840,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":726263,"Executor Run Time":1,"Executor CPU Time":1041036,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":127,"Index":126,"Attempt":0,"Partition ID":126,"Launch Time":1714515313958,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":126,"Index":125,"Attempt":0,"Partition ID":125,"Launch Time":1714515313954,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313959,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":824897,"Value":156818219,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":844,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":983960,"Value":724397886,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":316310,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":824897,"Executor Run Time":1,"Executor CPU Time":983960,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":128,"Index":127,"Attempt":0,"Partition ID":127,"Launch Time":1714515313963,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":127,"Index":126,"Attempt":0,"Partition ID":126,"Launch Time":1714515313958,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313964,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":939100,"Value":157757319,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":845,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1263027,"Value":725660913,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":318780,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":939100,"Executor Run Time":1,"Executor CPU Time":1263027,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":129,"Index":128,"Attempt":0,"Partition ID":128,"Launch Time":1714515313969,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":128,"Index":127,"Attempt":0,"Partition ID":127,"Launch Time":1714515313963,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313969,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":93,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1156410,"Value":158913729,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":847,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1623777,"Value":727284690,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":321293,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1156410,"Executor Run Time":2,"Executor CPU Time":1623777,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":130,"Index":129,"Attempt":0,"Partition ID":129,"Launch Time":1714515313972,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":129,"Index":128,"Attempt":0,"Partition ID":128,"Launch Time":1714515313969,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313972,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":626947,"Value":159540676,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":848,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":906274,"Value":728190964,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":323763,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":626947,"Executor Run Time":1,"Executor CPU Time":906274,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":131,"Index":130,"Attempt":0,"Partition ID":130,"Launch Time":1714515313976,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":130,"Index":129,"Attempt":0,"Partition ID":129,"Launch Time":1714515313972,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313976,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":823567,"Value":160364243,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":849,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1027667,"Value":729218631,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":326233,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":823567,"Executor Run Time":1,"Executor CPU Time":1027667,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":132,"Index":131,"Attempt":0,"Partition ID":131,"Launch Time":1714515313978,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":131,"Index":130,"Attempt":0,"Partition ID":130,"Launch Time":1714515313976,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313979,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":551212,"Value":160915455,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":850,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":825307,"Value":730043938,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":328703,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":551212,"Executor Run Time":1,"Executor CPU Time":825307,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":133,"Index":132,"Attempt":0,"Partition ID":132,"Launch Time":1714515313981,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":132,"Index":131,"Attempt":0,"Partition ID":131,"Launch Time":1714515313978,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313982,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":573631,"Value":161489086,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":851,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":795968,"Value":730839906,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":331173,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":573631,"Executor Run Time":1,"Executor CPU Time":795968,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":134,"Index":133,"Attempt":0,"Partition ID":133,"Launch Time":1714515313985,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":133,"Index":132,"Attempt":0,"Partition ID":132,"Launch Time":1714515313981,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313985,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":576891,"Value":162065977,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":852,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1450735,"Value":732290641,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":333643,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":576891,"Executor Run Time":1,"Executor CPU Time":1450735,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":135,"Index":134,"Attempt":0,"Partition ID":134,"Launch Time":1714515313988,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":134,"Index":133,"Attempt":0,"Partition ID":133,"Launch Time":1714515313985,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313989,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":631638,"Value":162697615,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":853,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":931491,"Value":733222132,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":336113,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":631638,"Executor Run Time":1,"Executor CPU Time":931491,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":136,"Index":135,"Attempt":0,"Partition ID":135,"Launch Time":1714515313992,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":135,"Index":134,"Attempt":0,"Partition ID":134,"Launch Time":1714515313988,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313992,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":806819,"Value":163504434,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":854,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1095173,"Value":734317305,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":338583,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":806819,"Executor Run Time":1,"Executor CPU Time":1095173,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":137,"Index":136,"Attempt":0,"Partition ID":136,"Launch Time":1714515313996,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":136,"Index":135,"Attempt":0,"Partition ID":135,"Launch Time":1714515313992,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515313997,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":922603,"Value":164427037,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":855,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1204927,"Value":735522232,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":341053,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":922603,"Executor Run Time":1,"Executor CPU Time":1204927,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":138,"Index":137,"Attempt":0,"Partition ID":137,"Launch Time":1714515314000,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":137,"Index":136,"Attempt":0,"Partition ID":136,"Launch Time":1714515313996,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314001,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":768371,"Value":165195408,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":856,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1149100,"Value":736671332,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":343523,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":768371,"Executor Run Time":1,"Executor CPU Time":1149100,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":139,"Index":138,"Attempt":0,"Partition ID":138,"Launch Time":1714515314003,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":138,"Index":137,"Attempt":0,"Partition ID":137,"Launch Time":1714515314000,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314004,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":607859,"Value":165803267,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":857,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":919392,"Value":737590724,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":345993,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":607859,"Executor Run Time":1,"Executor CPU Time":919392,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":140,"Index":139,"Attempt":0,"Partition ID":139,"Launch Time":1714515314007,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":139,"Index":138,"Attempt":0,"Partition ID":138,"Launch Time":1714515314003,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314007,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":639537,"Value":166442804,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":858,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":986540,"Value":738577264,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":348463,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":639537,"Executor Run Time":1,"Executor CPU Time":986540,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":141,"Index":140,"Attempt":0,"Partition ID":140,"Launch Time":1714515314010,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":140,"Index":139,"Attempt":0,"Partition ID":139,"Launch Time":1714515314007,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314010,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":596159,"Value":167038963,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":859,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":978630,"Value":739555894,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":350933,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":596159,"Executor Run Time":1,"Executor CPU Time":978630,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":142,"Index":141,"Attempt":0,"Partition ID":141,"Launch Time":1714515314013,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":141,"Index":140,"Attempt":0,"Partition ID":140,"Launch Time":1714515314010,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314014,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":645007,"Value":167683970,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":860,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":917242,"Value":740473136,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":353403,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":645007,"Executor Run Time":1,"Executor CPU Time":917242,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":143,"Index":142,"Attempt":0,"Partition ID":142,"Launch Time":1714515314017,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":142,"Index":141,"Attempt":0,"Partition ID":141,"Launch Time":1714515314013,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314018,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":609999,"Value":168293969,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":861,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1372788,"Value":741845924,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":355873,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":609999,"Executor Run Time":1,"Executor CPU Time":1372788,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":144,"Index":143,"Attempt":0,"Partition ID":143,"Launch Time":1714515314021,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":143,"Index":142,"Attempt":0,"Partition ID":142,"Launch Time":1714515314017,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314022,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":812799,"Value":169106768,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":862,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1107423,"Value":742953347,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":358343,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":812799,"Executor Run Time":1,"Executor CPU Time":1107423,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":145,"Index":144,"Attempt":0,"Partition ID":144,"Launch Time":1714515314025,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":144,"Index":143,"Attempt":0,"Partition ID":143,"Launch Time":1714515314021,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314025,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":637837,"Value":169744605,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":863,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1068845,"Value":744022192,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":360813,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":637837,"Executor Run Time":1,"Executor CPU Time":1068845,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":146,"Index":145,"Attempt":0,"Partition ID":145,"Launch Time":1714515314028,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":145,"Index":144,"Attempt":0,"Partition ID":144,"Launch Time":1714515314025,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314028,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":631658,"Value":170376263,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":864,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":976199,"Value":744998391,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":363283,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":631658,"Executor Run Time":1,"Executor CPU Time":976199,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":147,"Index":146,"Attempt":0,"Partition ID":146,"Launch Time":1714515314032,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":146,"Index":145,"Attempt":0,"Partition ID":145,"Launch Time":1714515314028,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314032,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":584260,"Value":170960523,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":865,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":873295,"Value":745871686,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":365753,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":584260,"Executor Run Time":1,"Executor CPU Time":873295,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":148,"Index":147,"Attempt":0,"Partition ID":147,"Launch Time":1714515314035,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":147,"Index":146,"Attempt":0,"Partition ID":146,"Launch Time":1714515314032,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314035,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":802038,"Value":171762561,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":866,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1137212,"Value":747008898,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":368223,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":802038,"Executor Run Time":1,"Executor CPU Time":1137212,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":149,"Index":148,"Attempt":0,"Partition ID":148,"Launch Time":1714515314039,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":148,"Index":147,"Attempt":0,"Partition ID":147,"Launch Time":1714515314035,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314039,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":641387,"Value":172403948,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":867,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":963470,"Value":747972368,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":370693,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":641387,"Executor Run Time":1,"Executor CPU Time":963470,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":150,"Index":149,"Attempt":0,"Partition ID":149,"Launch Time":1714515314042,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":149,"Index":148,"Attempt":0,"Partition ID":148,"Launch Time":1714515314039,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314042,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":729433,"Value":173133381,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":868,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":967339,"Value":748939707,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":373163,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":729433,"Executor Run Time":1,"Executor CPU Time":967339,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":151,"Index":150,"Attempt":0,"Partition ID":150,"Launch Time":1714515314045,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":150,"Index":149,"Attempt":0,"Partition ID":149,"Launch Time":1714515314042,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314046,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":703994,"Value":173837375,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":869,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":945361,"Value":749885068,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":375633,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":703994,"Executor Run Time":1,"Executor CPU Time":945361,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":152,"Index":151,"Attempt":0,"Partition ID":151,"Launch Time":1714515314050,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":151,"Index":150,"Attempt":0,"Partition ID":150,"Launch Time":1714515314045,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314050,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":780510,"Value":174617885,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":870,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1141422,"Value":751026490,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":378103,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":780510,"Executor Run Time":1,"Executor CPU Time":1141422,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":153,"Index":152,"Attempt":0,"Partition ID":152,"Launch Time":1714515314055,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":152,"Index":151,"Attempt":0,"Partition ID":151,"Launch Time":1714515314050,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314055,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":854766,"Value":175472651,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":872,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1622217,"Value":752648707,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":380573,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":854766,"Executor Run Time":2,"Executor CPU Time":1622217,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":154,"Index":153,"Attempt":0,"Partition ID":153,"Launch Time":1714515314059,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":153,"Index":152,"Attempt":0,"Partition ID":152,"Launch Time":1714515314055,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314059,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":826198,"Value":176298849,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":873,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1187429,"Value":753836136,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":383043,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":826198,"Executor Run Time":1,"Executor CPU Time":1187429,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":155,"Index":154,"Attempt":0,"Partition ID":154,"Launch Time":1714515314064,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":154,"Index":153,"Attempt":0,"Partition ID":153,"Launch Time":1714515314059,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314064,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":970640,"Value":177269489,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":874,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1326182,"Value":755162318,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":385513,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":970640,"Executor Run Time":1,"Executor CPU Time":1326182,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":156,"Index":155,"Attempt":0,"Partition ID":155,"Launch Time":1714515314070,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":155,"Index":154,"Attempt":0,"Partition ID":154,"Launch Time":1714515314064,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314070,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":918443,"Value":178187932,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":876,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":2274903,"Value":757437221,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":387983,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":918443,"Executor Run Time":2,"Executor CPU Time":2274903,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":157,"Index":156,"Attempt":0,"Partition ID":156,"Launch Time":1714515314075,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":156,"Index":155,"Attempt":0,"Partition ID":155,"Launch Time":1714515314070,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314076,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":94,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1070485,"Value":179258417,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":877,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1442705,"Value":758879926,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":390496,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1070485,"Executor Run Time":1,"Executor CPU Time":1442705,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":158,"Index":157,"Attempt":0,"Partition ID":157,"Launch Time":1714515314081,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":157,"Index":156,"Attempt":0,"Partition ID":156,"Launch Time":1714515314075,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314081,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":949221,"Value":180207638,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":878,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1569269,"Value":760449195,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":392966,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":949221,"Executor Run Time":1,"Executor CPU Time":1569269,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":159,"Index":158,"Attempt":0,"Partition ID":158,"Launch Time":1714515314087,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":158,"Index":157,"Attempt":0,"Partition ID":157,"Launch Time":1714515314081,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314087,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1576038,"Value":181783676,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":880,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1571309,"Value":762020504,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":395479,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1576038,"Executor Run Time":2,"Executor CPU Time":1571309,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":160,"Index":159,"Attempt":0,"Partition ID":159,"Launch Time":1714515314092,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":159,"Index":158,"Attempt":0,"Partition ID":158,"Launch Time":1714515314087,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314092,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":952921,"Value":182736597,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":881,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1309773,"Value":763330277,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":397949,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":952921,"Executor Run Time":1,"Executor CPU Time":1309773,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":161,"Index":160,"Attempt":0,"Partition ID":160,"Launch Time":1714515314096,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":160,"Index":159,"Attempt":0,"Partition ID":159,"Launch Time":1714515314092,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314097,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":877674,"Value":183614271,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":882,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1217419,"Value":764547696,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":400419,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":877674,"Executor Run Time":1,"Executor CPU Time":1217419,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":162,"Index":161,"Attempt":0,"Partition ID":161,"Launch Time":1714515314101,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":161,"Index":160,"Attempt":0,"Partition ID":160,"Launch Time":1714515314096,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314101,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":781859,"Value":184396130,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":883,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1396759,"Value":765944455,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":402889,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":781859,"Executor Run Time":1,"Executor CPU Time":1396759,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":163,"Index":162,"Attempt":0,"Partition ID":162,"Launch Time":1714515314105,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":162,"Index":161,"Attempt":0,"Partition ID":161,"Launch Time":1714515314101,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314106,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":992449,"Value":185388579,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":884,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1254716,"Value":767199171,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":405359,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":992449,"Executor Run Time":1,"Executor CPU Time":1254716,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":164,"Index":163,"Attempt":0,"Partition ID":163,"Launch Time":1714515314110,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":163,"Index":162,"Attempt":0,"Partition ID":162,"Launch Time":1714515314105,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314110,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":834687,"Value":186223266,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":885,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1237255,"Value":768436426,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":407829,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":834687,"Executor Run Time":1,"Executor CPU Time":1237255,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":165,"Index":164,"Attempt":0,"Partition ID":164,"Launch Time":1714515314114,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":164,"Index":163,"Attempt":0,"Partition ID":163,"Launch Time":1714515314110,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314114,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":896793,"Value":187120059,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":886,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1094445,"Value":769530871,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":410299,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":896793,"Executor Run Time":1,"Executor CPU Time":1094445,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":166,"Index":165,"Attempt":0,"Partition ID":165,"Launch Time":1714515314118,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":165,"Index":164,"Attempt":0,"Partition ID":164,"Launch Time":1714515314114,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314119,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":819797,"Value":187939856,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":888,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1663526,"Value":771194397,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":412769,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":819797,"Executor Run Time":2,"Executor CPU Time":1663526,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":167,"Index":166,"Attempt":0,"Partition ID":166,"Launch Time":1714515314125,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":166,"Index":165,"Attempt":0,"Partition ID":165,"Launch Time":1714515314118,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314125,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":883624,"Value":188823480,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":889,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1237056,"Value":772431453,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":415239,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":883624,"Executor Run Time":1,"Executor CPU Time":1237056,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":168,"Index":167,"Attempt":0,"Partition ID":167,"Launch Time":1714515314130,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":167,"Index":166,"Attempt":0,"Partition ID":166,"Launch Time":1714515314125,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314130,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":904394,"Value":189727874,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":890,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1213206,"Value":773644659,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":417709,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":904394,"Executor Run Time":1,"Executor CPU Time":1213206,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":169,"Index":168,"Attempt":0,"Partition ID":168,"Launch Time":1714515314134,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":168,"Index":167,"Attempt":0,"Partition ID":167,"Launch Time":1714515314130,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314134,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":824548,"Value":190552422,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":891,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1154781,"Value":774799440,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":420179,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":824548,"Executor Run Time":1,"Executor CPU Time":1154781,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":170,"Index":169,"Attempt":0,"Partition ID":169,"Launch Time":1714515314138,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":169,"Index":168,"Attempt":0,"Partition ID":168,"Launch Time":1714515314134,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314138,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":776331,"Value":191328753,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":892,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1053404,"Value":775852844,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":422649,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":776331,"Executor Run Time":1,"Executor CPU Time":1053404,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":171,"Index":170,"Attempt":0,"Partition ID":170,"Launch Time":1714515314141,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":170,"Index":169,"Attempt":0,"Partition ID":169,"Launch Time":1714515314138,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314141,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":732652,"Value":192061405,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":893,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":966011,"Value":776818855,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":425119,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":732652,"Executor Run Time":1,"Executor CPU Time":966011,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":172,"Index":171,"Attempt":0,"Partition ID":171,"Launch Time":1714515314145,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":171,"Index":170,"Attempt":0,"Partition ID":170,"Launch Time":1714515314141,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314145,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":641377,"Value":192702782,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":894,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":928722,"Value":777747577,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":427589,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":641377,"Executor Run Time":1,"Executor CPU Time":928722,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":173,"Index":172,"Attempt":0,"Partition ID":172,"Launch Time":1714515314147,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":172,"Index":171,"Attempt":0,"Partition ID":171,"Launch Time":1714515314145,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314148,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":598089,"Value":193300871,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":895,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":819908,"Value":778567485,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":430059,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":598089,"Executor Run Time":1,"Executor CPU Time":819908,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":174,"Index":173,"Attempt":0,"Partition ID":173,"Launch Time":1714515314151,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":173,"Index":172,"Attempt":0,"Partition ID":172,"Launch Time":1714515314147,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314151,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":554061,"Value":193854932,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":896,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1092444,"Value":779659929,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":432529,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":554061,"Executor Run Time":1,"Executor CPU Time":1092444,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":175,"Index":174,"Attempt":0,"Partition ID":174,"Launch Time":1714515314154,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":174,"Index":173,"Attempt":0,"Partition ID":173,"Launch Time":1714515314151,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314154,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":608819,"Value":194463751,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":897,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":880445,"Value":780540374,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":434999,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":608819,"Executor Run Time":1,"Executor CPU Time":880445,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":176,"Index":175,"Attempt":0,"Partition ID":175,"Launch Time":1714515314157,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":175,"Index":174,"Attempt":0,"Partition ID":174,"Launch Time":1714515314154,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314157,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":593959,"Value":195057710,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":898,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":782341,"Value":781322715,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":437469,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":593959,"Executor Run Time":1,"Executor CPU Time":782341,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":177,"Index":176,"Attempt":0,"Partition ID":176,"Launch Time":1714515314160,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":176,"Index":175,"Attempt":0,"Partition ID":175,"Launch Time":1714515314157,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314160,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":658837,"Value":195716547,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":899,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":754821,"Value":782077536,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":439939,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":658837,"Executor Run Time":1,"Executor CPU Time":754821,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":178,"Index":177,"Attempt":0,"Partition ID":177,"Launch Time":1714515314163,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":177,"Index":176,"Attempt":0,"Partition ID":176,"Launch Time":1714515314160,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314163,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":581710,"Value":196298257,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":900,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":763450,"Value":782840986,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":442409,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":581710,"Executor Run Time":1,"Executor CPU Time":763450,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":179,"Index":178,"Attempt":0,"Partition ID":178,"Launch Time":1714515314165,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":178,"Index":177,"Attempt":0,"Partition ID":177,"Launch Time":1714515314163,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314166,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":577551,"Value":196875808,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":901,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":750861,"Value":783591847,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":444879,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":577551,"Executor Run Time":1,"Executor CPU Time":750861,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":180,"Index":179,"Attempt":0,"Partition ID":179,"Launch Time":1714515314169,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":179,"Index":178,"Attempt":0,"Partition ID":178,"Launch Time":1714515314165,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314169,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":740253,"Value":197616061,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":902,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":961570,"Value":784553417,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":447349,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":740253,"Executor Run Time":1,"Executor CPU Time":961570,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":181,"Index":180,"Attempt":0,"Partition ID":180,"Launch Time":1714515314173,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":180,"Index":179,"Attempt":0,"Partition ID":179,"Launch Time":1714515314169,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314173,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":590499,"Value":198206560,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":903,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":787140,"Value":785340557,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":449819,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":590499,"Executor Run Time":1,"Executor CPU Time":787140,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":182,"Index":181,"Attempt":0,"Partition ID":181,"Launch Time":1714515314176,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":181,"Index":180,"Attempt":0,"Partition ID":180,"Launch Time":1714515314173,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314176,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":605449,"Value":198812009,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":904,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":823277,"Value":786163834,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":452289,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":605449,"Executor Run Time":1,"Executor CPU Time":823277,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":183,"Index":182,"Attempt":0,"Partition ID":182,"Launch Time":1714515314179,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":182,"Index":181,"Attempt":0,"Partition ID":181,"Launch Time":1714515314176,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314179,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":603019,"Value":199415028,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":791519,"Value":786955353,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":454716,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":603019,"Executor Run Time":0,"Executor CPU Time":791519,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":184,"Index":183,"Attempt":0,"Partition ID":183,"Launch Time":1714515314182,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":183,"Index":182,"Attempt":0,"Partition ID":182,"Launch Time":1714515314179,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314182,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":592240,"Value":200007268,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":753101,"Value":787708454,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2427,"Value":457143,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":592240,"Executor Run Time":0,"Executor CPU Time":753101,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":185,"Index":184,"Attempt":0,"Partition ID":184,"Launch Time":1714515314193,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":184,"Index":183,"Attempt":0,"Partition ID":183,"Launch Time":1714515314182,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314193,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":96,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1100832,"Value":201108100,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":905,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1403377,"Value":789111831,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":459656,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1100832,"Executor Run Time":1,"Executor CPU Time":1403377,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":186,"Index":185,"Attempt":0,"Partition ID":185,"Launch Time":1714515314198,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":185,"Index":184,"Attempt":0,"Partition ID":184,"Launch Time":1714515314193,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314199,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":97,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":999499,"Value":202107599,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":2,"Value":907,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1397617,"Value":790509448,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":462169,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":999499,"Executor Run Time":2,"Executor CPU Time":1397617,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":187,"Index":186,"Attempt":0,"Partition ID":186,"Launch Time":1714515314203,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":186,"Index":185,"Attempt":0,"Partition ID":185,"Launch Time":1714515314198,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314204,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":98,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1150770,"Value":203258369,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":908,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1300725,"Value":791810173,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2513,"Value":464682,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1150770,"Executor Run Time":1,"Executor CPU Time":1300725,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":188,"Index":187,"Attempt":0,"Partition ID":187,"Launch Time":1714515314207,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":187,"Index":186,"Attempt":0,"Partition ID":186,"Launch Time":1714515314203,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314207,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":716772,"Value":203975141,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":909,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1018547,"Value":792828720,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":467152,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":716772,"Executor Run Time":1,"Executor CPU Time":1018547,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":189,"Index":188,"Attempt":0,"Partition ID":188,"Launch Time":1714515314210,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":188,"Index":187,"Attempt":0,"Partition ID":187,"Launch Time":1714515314207,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314210,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":659616,"Value":204634757,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":910,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":885864,"Value":793714584,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":469622,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":659616,"Executor Run Time":1,"Executor CPU Time":885864,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":190,"Index":189,"Attempt":0,"Partition ID":189,"Launch Time":1714515314213,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":189,"Index":188,"Attempt":0,"Partition ID":188,"Launch Time":1714515314210,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314213,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":687594,"Value":205322351,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":911,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":837917,"Value":794552501,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":472092,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":687594,"Executor Run Time":1,"Executor CPU Time":837917,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":191,"Index":190,"Attempt":0,"Partition ID":190,"Launch Time":1714515314216,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":190,"Index":189,"Attempt":0,"Partition ID":189,"Launch Time":1714515314213,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314217,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":598810,"Value":205921161,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":912,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":775449,"Value":795327950,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":474562,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":598810,"Executor Run Time":1,"Executor CPU Time":775449,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":192,"Index":191,"Attempt":0,"Partition ID":191,"Launch Time":1714515314220,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":191,"Index":190,"Attempt":0,"Partition ID":190,"Launch Time":1714515314216,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314220,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":702923,"Value":206624084,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":913,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":887525,"Value":796215475,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":477032,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":702923,"Executor Run Time":1,"Executor CPU Time":887525,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":193,"Index":192,"Attempt":0,"Partition ID":192,"Launch Time":1714515314223,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":192,"Index":191,"Attempt":0,"Partition ID":191,"Launch Time":1714515314220,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314223,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":675035,"Value":207299119,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":914,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":969190,"Value":797184665,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":479502,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":675035,"Executor Run Time":1,"Executor CPU Time":969190,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":194,"Index":193,"Attempt":0,"Partition ID":193,"Launch Time":1714515314226,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":193,"Index":192,"Attempt":0,"Partition ID":192,"Launch Time":1714515314223,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314226,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":619099,"Value":207918218,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":915,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":800618,"Value":797985283,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":481972,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":619099,"Executor Run Time":1,"Executor CPU Time":800618,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":195,"Index":194,"Attempt":0,"Partition ID":194,"Launch Time":1714515314229,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":194,"Index":193,"Attempt":0,"Partition ID":193,"Launch Time":1714515314226,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314229,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":622589,"Value":208540807,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":916,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":824866,"Value":798810149,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":484442,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":622589,"Executor Run Time":1,"Executor CPU Time":824866,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":196,"Index":195,"Attempt":0,"Partition ID":195,"Launch Time":1714515314233,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":195,"Index":194,"Attempt":0,"Partition ID":194,"Launch Time":1714515314229,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314233,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":880954,"Value":209421761,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":917,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1116743,"Value":799926892,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":486912,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":880954,"Executor Run Time":1,"Executor CPU Time":1116743,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":197,"Index":196,"Attempt":0,"Partition ID":196,"Launch Time":1714515314236,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":196,"Index":195,"Attempt":0,"Partition ID":195,"Launch Time":1714515314233,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314236,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":649677,"Value":210071438,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":918,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":882805,"Value":800809697,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":489382,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":649677,"Executor Run Time":1,"Executor CPU Time":882805,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":198,"Index":197,"Attempt":0,"Partition ID":197,"Launch Time":1714515314239,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":197,"Index":196,"Attempt":0,"Partition ID":196,"Launch Time":1714515314236,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314239,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":783420,"Value":210854858,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":919,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":995569,"Value":801805266,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":491852,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":783420,"Executor Run Time":1,"Executor CPU Time":995569,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":199,"Index":198,"Attempt":0,"Partition ID":198,"Launch Time":1714515314243,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":198,"Index":197,"Attempt":0,"Partition ID":197,"Launch Time":1714515314239,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314243,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":736442,"Value":211591300,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":920,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":1043406,"Value":802848672,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":494322,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":736442,"Executor Run Time":1,"Executor CPU Time":1043406,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":200,"Index":199,"Attempt":0,"Partition ID":199,"Launch Time":1714515314246,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":199,"Index":198,"Attempt":0,"Partition ID":198,"Launch Time":1714515314243,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314246,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":608389,"Value":212199689,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":921,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":815468,"Value":803664140,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":496792,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":608389,"Executor Run Time":1,"Executor CPU Time":815468,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":200,"Index":199,"Attempt":0,"Partition ID":199,"Launch Time":1714515314246,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314249,"Failed":false,"Killed":false,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":804598,"Value":213004287,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":1,"Value":922,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":844287,"Value":804508427,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2470,"Value":499262,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":804598,"Executor Run Time":1,"Executor CPU Time":844287,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"processLine at CliDriver.java:336","Number of Tasks":200,"RDD Info":[{"RDD ID":3,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"4\",\"name\":\"Exchange\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"UNORDERED","Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[1],"Details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1714515312577,"Completion Time":1714515314250,"Accumulables":[{"ID":5,"Name":"local blocks read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"local bytes read","Value":"95","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":9,"Name":"fetch wait time","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":10,"Name":"records read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Value":98,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Value":213004287,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Value":922,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Value":804508427,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Value":499262,"Internal":true,"Count Failed Values":true},{"ID":45,"Name":"internal.metrics.jvmGCTime","Value":62,"Internal":true,"Count Failed Values":true},{"ID":46,"Name":"internal.metrics.resultSerializationTime","Value":2,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":95,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.read.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.output.bytesWritten","Value":932,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.output.recordsWritten","Value":1,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":1,"Completion Time":1714515314252,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":4,"time":1714515314431} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":5,"description":"\nINSERT INTO local.developers VALUES(\"krisnova\", \"Kris\", \"Nova\")","details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nCommandResult (1)\n +- AppendData (6)\n +- AdaptiveSparkPlan (5)\n +- == Final Plan ==\n ShuffleQueryStage (4), Statistics(sizeInBytes=56.0 B, rowCount=1)\n +- Exchange (3)\n +- LocalTableScan (2)\n +- == Initial Plan ==\n Exchange (3)\n +- LocalTableScan (2)\n\n\n(1) CommandResult\nOutput: []\nArguments: \n\n(2) LocalTableScan\nOutput [3]: [username#6, firstname#7, lastname#8]\nArguments: [username#6, firstname#7, lastname#8]\n\n(3) Exchange\nInput [3]: [username#6, firstname#7, lastname#8]\nArguments: hashpartitioning(username#6, 200), REPARTITION_BY_NUM, [plan_id=21]\n\n(4) ShuffleQueryStage\nOutput [3]: [username#6, firstname#7, lastname#8]\nArguments: 0\n\n(5) AdaptiveSparkPlan\nOutput [3]: [username#6, firstname#7, lastname#8]\nArguments: isFinalPlan=true\n\n(6) AppendData\nInput [3]: [username#6, firstname#7, lastname#8]\nArguments: org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$2453/0x00007505ed0efbf0@418f2af3, IcebergWrite(table=local.developers, format=PARQUET)\n\n","sparkPlanInfo":{"nodeName":"CommandResult","simpleString":"CommandResult ","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":65,"metricType":"sum"}]},"time":1714515314438,"modifiedConfigs":{}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":5,"time":1714515314438} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":6,"description":"\nINSERT INTO local.projects VALUES(\"krisnova\", \"aurae\")","details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nAppendData (4)\n+- AdaptiveSparkPlan (3)\n +- Exchange (2)\n +- LocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [2]: [creator#13, projectname#14]\nArguments: [creator#13, projectname#14]\n\n(2) Exchange\nInput [2]: [creator#13, projectname#14]\nArguments: hashpartitioning(creator#13, 200), REPARTITION_BY_NUM, [plan_id=42]\n\n(3) AdaptiveSparkPlan\nOutput [2]: [creator#13, projectname#14]\nArguments: isFinalPlan=false\n\n(4) AppendData\nInput [2]: [creator#13, projectname#14]\nArguments: org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$2453/0x00007505ed0efbf0@3b3d7eb6, IcebergWrite(table=local.projects, format=PARQUET)\n\n","sparkPlanInfo":{"nodeName":"AppendData","simpleString":"AppendData org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$2453/0x00007505ed0efbf0@3b3d7eb6, IcebergWrite(table=local.projects, format=PARQUET)","children":[{"nodeName":"AdaptiveSparkPlan","simpleString":"AdaptiveSparkPlan isFinalPlan=false","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(creator#13, 200), REPARTITION_BY_NUM, [plan_id=42]","children":[{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [creator#13, projectname#14]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":78,"metricType":"sum"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":76,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":77,"metricType":"nsTiming"},{"name":"records read","accumulatorId":74,"metricType":"sum"},{"name":"local bytes read","accumulatorId":72,"metricType":"size"},{"name":"fetch wait time","accumulatorId":73,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":70,"metricType":"size"},{"name":"local blocks read","accumulatorId":69,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":68,"metricType":"sum"},{"name":"data size","accumulatorId":66,"metricType":"size"},{"name":"number of partitions","accumulatorId":67,"metricType":"sum"},{"name":"remote bytes read to disk","accumulatorId":71,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":75,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]},"time":1714515314453,"modifiedConfigs":{}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate","executionId":6,"physicalPlanDescription":"== Physical Plan ==\nAppendData (5)\n+- AdaptiveSparkPlan (4)\n +- == Current Plan ==\n ShuffleQueryStage (3)\n +- Exchange (2)\n +- LocalTableScan (1)\n +- == Initial Plan ==\n Exchange (2)\n +- LocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [2]: [creator#13, projectname#14]\nArguments: [creator#13, projectname#14]\n\n(2) Exchange\nInput [2]: [creator#13, projectname#14]\nArguments: hashpartitioning(creator#13, 200), REPARTITION_BY_NUM, [plan_id=42]\n\n(3) ShuffleQueryStage\nOutput [2]: [creator#13, projectname#14]\nArguments: 0\n\n(4) AdaptiveSparkPlan\nOutput [2]: [creator#13, projectname#14]\nArguments: isFinalPlan=false\n\n(5) AppendData\nInput [2]: [creator#13, projectname#14]\nArguments: org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$2453/0x00007505ed0efbf0@3b3d7eb6, IcebergWrite(table=local.projects, format=PARQUET)\n\n","sparkPlanInfo":{"nodeName":"AppendData","simpleString":"AppendData org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$2453/0x00007505ed0efbf0@3b3d7eb6, IcebergWrite(table=local.projects, format=PARQUET)","children":[{"nodeName":"AdaptiveSparkPlan","simpleString":"AdaptiveSparkPlan isFinalPlan=false","children":[{"nodeName":"ShuffleQueryStage","simpleString":"ShuffleQueryStage 0","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(creator#13, 200), REPARTITION_BY_NUM, [plan_id=42]","children":[{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [creator#13, projectname#14]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":78,"metricType":"sum"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":76,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":77,"metricType":"nsTiming"},{"name":"records read","accumulatorId":74,"metricType":"sum"},{"name":"local bytes read","accumulatorId":72,"metricType":"size"},{"name":"fetch wait time","accumulatorId":73,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":70,"metricType":"size"},{"name":"local blocks read","accumulatorId":69,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":68,"metricType":"sum"},{"name":"data size","accumulatorId":66,"metricType":"size"},{"name":"number of partitions","accumulatorId":67,"metricType":"sum"},{"name":"remote bytes read to disk","accumulatorId":71,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":75,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":6,"accumUpdates":[[67,200]]} +{"Event":"SparkListenerJobStart","Job ID":2,"Submission Time":1714515314466,"Stage Infos":[{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"processLine at CliDriver.java:336","Number of Tasks":1,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"Exchange\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"LocalTableScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"11\",\"name\":\"LocalTableScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[3],"Properties":{"spark.dynamicAllocation.shuffleTracking.enabled":"true","spark.sql.warehouse.dir":"file:/home/holden/repos/sparklens/spark-warehouse","spark.executor.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.driver.host":"10.69.200.180","spark.eventLog.enabled":"true","spark.driver.port":"37323","__fetch_continuous_blocks_in_batch_enabled":"true","spark.sql.catalog.local.type":"hadoop","spark.jars":"","spark.sql.catalog.spark_catalog":"org.apache.iceberg.spark.SparkSessionCatalog","spark.kubernetes.authenticate.submission.caCertFile":"/var/snap/microk8s/current/certs/ca.crt","spark.sql.datetime.java8API.enabled":"true","spark.sql.hive.version":"2.3.9","spark.app.name":"fresh-kube","spark.rdd.scope":"{\"id\":\"10\",\"name\":\"Exchange\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.submitTime":"1714515303940","spark.job.description":"\nINSERT INTO local.projects VALUES(\"krisnova\", \"aurae\")","spark.kubernetes.container.image":"localhost:32000/local-spark/spark:magic","spark.app.startTime":"1714515304235","spark.executor.id":"driver","spark.sql.catalog.local.warehouse":"/tmp/warehouse","spark.driver.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.sql.catalog.local":"org.apache.iceberg.spark.SparkCatalog","spark.submit.deployMode":"client","spark.master":"k8s://https://127.0.0.1:16443","spark.kubernetes.executor.podNamePrefix":"fresh-kube-bb0c5b8f311360be","spark.dynamicAllocation.enabled":"true","spark.sql.execution.id":"6","spark.sql.catalogImplementation":"hive","spark.kubernetes.authenticate.caCertFile":"/var/snap/microk8s/current/certs/ca.crt","spark.sql.catalog.spark_catalog.type":"hive","spark.app.id":"spark-a8d39974ec7b451a92b303fa953434e4"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"processLine at CliDriver.java:336","Number of Tasks":1,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"Exchange\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"LocalTableScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"11\",\"name\":\"LocalTableScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1714515314467,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.dynamicAllocation.shuffleTracking.enabled":"true","spark.sql.warehouse.dir":"file:/home/holden/repos/sparklens/spark-warehouse","spark.executor.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.driver.host":"10.69.200.180","spark.eventLog.enabled":"true","spark.driver.port":"37323","__fetch_continuous_blocks_in_batch_enabled":"true","spark.sql.catalog.local.type":"hadoop","spark.jars":"","spark.sql.catalog.spark_catalog":"org.apache.iceberg.spark.SparkSessionCatalog","spark.kubernetes.authenticate.submission.caCertFile":"/var/snap/microk8s/current/certs/ca.crt","spark.sql.datetime.java8API.enabled":"true","spark.sql.hive.version":"2.3.9","spark.app.name":"fresh-kube","spark.rdd.scope":"{\"id\":\"10\",\"name\":\"Exchange\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.submitTime":"1714515303940","spark.job.description":"\nINSERT INTO local.projects VALUES(\"krisnova\", \"aurae\")","spark.kubernetes.container.image":"localhost:32000/local-spark/spark:magic","spark.app.startTime":"1714515304235","spark.executor.id":"driver","spark.sql.catalog.local.warehouse":"/tmp/warehouse","spark.driver.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.sql.catalog.local":"org.apache.iceberg.spark.SparkCatalog","spark.submit.deployMode":"client","spark.master":"k8s://https://127.0.0.1:16443","spark.kubernetes.executor.podNamePrefix":"fresh-kube-bb0c5b8f311360be","spark.dynamicAllocation.enabled":"true","spark.sql.execution.id":"6","spark.sql.catalogImplementation":"hive","spark.kubernetes.authenticate.caCertFile":"/var/snap/microk8s/current/certs/ca.crt","spark.sql.catalog.spark_catalog.type":"hive","spark.app.id":"spark-a8d39974ec7b451a92b303fa953434e4"}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":201,"Index":0,"Attempt":0,"Partition ID":0,"Launch Time":1714515314471,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":201,"Index":0,"Attempt":0,"Partition ID":0,"Launch Time":1714515314471,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314488,"Failed":false,"Killed":false,"Accumulables":[{"ID":66,"Name":"data size","Update":"40","Value":"40","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":75,"Name":"shuffle bytes written","Update":"84","Value":"84","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":76,"Name":"shuffle records written","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":77,"Name":"shuffle write time","Update":"3970063","Value":"3970063","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":78,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":79,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":6,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3582555,"Value":3582555,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.executorRunTime","Update":7,"Value":7,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.executorCpuTime","Update":7896555,"Value":7896555,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.resultSize","Update":1822,"Value":1822,"Internal":true,"Count Failed Values":true},{"ID":97,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":84,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":98,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":99,"Name":"internal.metrics.shuffle.write.writeTime","Update":3970063,"Value":3970063,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":3582555,"Executor Run Time":7,"Executor CPU Time":7896555,"Peak Execution Memory":0,"Result Size":1822,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":84,"Shuffle Write Time":3970063,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"processLine at CliDriver.java:336","Number of Tasks":1,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"Exchange\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"LocalTableScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"11\",\"name\":\"LocalTableScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1714515314467,"Completion Time":1714515314489,"Accumulables":[{"ID":66,"Name":"data size","Value":"40","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":75,"Name":"shuffle bytes written","Value":"84","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":76,"Name":"shuffle records written","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":77,"Name":"shuffle write time","Value":"3970063","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":78,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":79,"Name":"internal.metrics.executorDeserializeTime","Value":6,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.executorDeserializeCpuTime","Value":3582555,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.executorRunTime","Value":7,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.executorCpuTime","Value":7896555,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.resultSize","Value":1822,"Internal":true,"Count Failed Values":true},{"ID":97,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":84,"Internal":true,"Count Failed Values":true},{"ID":98,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":1,"Internal":true,"Count Failed Values":true},{"ID":99,"Name":"internal.metrics.shuffle.write.writeTime","Value":3970063,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":2,"Completion Time":1714515314489,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate","executionId":6,"physicalPlanDescription":"== Physical Plan ==\nAppendData (5)\n+- AdaptiveSparkPlan (4)\n +- == Final Plan ==\n ShuffleQueryStage (3), Statistics(sizeInBytes=40.0 B, rowCount=1)\n +- Exchange (2)\n +- LocalTableScan (1)\n +- == Initial Plan ==\n Exchange (2)\n +- LocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [2]: [creator#13, projectname#14]\nArguments: [creator#13, projectname#14]\n\n(2) Exchange\nInput [2]: [creator#13, projectname#14]\nArguments: hashpartitioning(creator#13, 200), REPARTITION_BY_NUM, [plan_id=42]\n\n(3) ShuffleQueryStage\nOutput [2]: [creator#13, projectname#14]\nArguments: 0\n\n(4) AdaptiveSparkPlan\nOutput [2]: [creator#13, projectname#14]\nArguments: isFinalPlan=true\n\n(5) AppendData\nInput [2]: [creator#13, projectname#14]\nArguments: org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$2453/0x00007505ed0efbf0@3b3d7eb6, IcebergWrite(table=local.projects, format=PARQUET)\n\n","sparkPlanInfo":{"nodeName":"AppendData","simpleString":"AppendData org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$2453/0x00007505ed0efbf0@3b3d7eb6, IcebergWrite(table=local.projects, format=PARQUET)","children":[{"nodeName":"AdaptiveSparkPlan","simpleString":"AdaptiveSparkPlan isFinalPlan=true","children":[{"nodeName":"ShuffleQueryStage","simpleString":"ShuffleQueryStage 0","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(creator#13, 200), REPARTITION_BY_NUM, [plan_id=42]","children":[{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [creator#13, projectname#14]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":78,"metricType":"sum"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":76,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":77,"metricType":"nsTiming"},{"name":"records read","accumulatorId":74,"metricType":"sum"},{"name":"local bytes read","accumulatorId":72,"metricType":"size"},{"name":"fetch wait time","accumulatorId":73,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":70,"metricType":"size"},{"name":"local blocks read","accumulatorId":69,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":68,"metricType":"sum"},{"name":"data size","accumulatorId":66,"metricType":"size"},{"name":"number of partitions","accumulatorId":67,"metricType":"sum"},{"name":"remote bytes read to disk","accumulatorId":71,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":75,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}} +{"Event":"SparkListenerJobStart","Job ID":3,"Submission Time":1714515314499,"Stage Infos":[{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"processLine at CliDriver.java:336","Number of Tasks":200,"RDD Info":[{"RDD ID":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"13\",\"name\":\"Exchange\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"UNORDERED","Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[4],"Details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"processLine at CliDriver.java:336","Number of Tasks":1,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"Exchange\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"LocalTableScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"11\",\"name\":\"LocalTableScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[5,4],"Properties":{"spark.dynamicAllocation.shuffleTracking.enabled":"true","spark.sql.warehouse.dir":"file:/home/holden/repos/sparklens/spark-warehouse","spark.executor.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.driver.host":"10.69.200.180","spark.eventLog.enabled":"true","spark.driver.port":"37323","__fetch_continuous_blocks_in_batch_enabled":"true","spark.sql.catalog.local.type":"hadoop","spark.jars":"","spark.sql.catalog.spark_catalog":"org.apache.iceberg.spark.SparkSessionCatalog","spark.kubernetes.authenticate.submission.caCertFile":"/var/snap/microk8s/current/certs/ca.crt","spark.sql.datetime.java8API.enabled":"true","spark.sql.hive.version":"2.3.9","spark.app.name":"fresh-kube","spark.submit.pyFiles":"","spark.app.submitTime":"1714515303940","spark.job.description":"\nINSERT INTO local.projects VALUES(\"krisnova\", \"aurae\")","spark.kubernetes.container.image":"localhost:32000/local-spark/spark:magic","spark.app.startTime":"1714515304235","spark.executor.id":"driver","spark.sql.catalog.local.warehouse":"/tmp/warehouse","spark.driver.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.sql.catalog.local":"org.apache.iceberg.spark.SparkCatalog","spark.submit.deployMode":"client","spark.master":"k8s://https://127.0.0.1:16443","spark.kubernetes.executor.podNamePrefix":"fresh-kube-bb0c5b8f311360be","spark.dynamicAllocation.enabled":"true","spark.sql.execution.id":"6","spark.sql.catalogImplementation":"hive","spark.kubernetes.authenticate.caCertFile":"/var/snap/microk8s/current/certs/ca.crt","spark.sql.catalog.spark_catalog.type":"hive","spark.app.id":"spark-a8d39974ec7b451a92b303fa953434e4"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"processLine at CliDriver.java:336","Number of Tasks":200,"RDD Info":[{"RDD ID":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"13\",\"name\":\"Exchange\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"UNORDERED","Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[4],"Details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1714515314503,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.dynamicAllocation.shuffleTracking.enabled":"true","spark.sql.warehouse.dir":"file:/home/holden/repos/sparklens/spark-warehouse","spark.executor.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.driver.host":"10.69.200.180","spark.eventLog.enabled":"true","spark.driver.port":"37323","__fetch_continuous_blocks_in_batch_enabled":"true","spark.sql.catalog.local.type":"hadoop","spark.jars":"","spark.sql.catalog.spark_catalog":"org.apache.iceberg.spark.SparkSessionCatalog","spark.kubernetes.authenticate.submission.caCertFile":"/var/snap/microk8s/current/certs/ca.crt","spark.sql.datetime.java8API.enabled":"true","spark.sql.hive.version":"2.3.9","spark.app.name":"fresh-kube","spark.submit.pyFiles":"","spark.app.submitTime":"1714515303940","spark.job.description":"\nINSERT INTO local.projects VALUES(\"krisnova\", \"aurae\")","spark.kubernetes.container.image":"localhost:32000/local-spark/spark:magic","spark.app.startTime":"1714515304235","spark.executor.id":"driver","spark.sql.catalog.local.warehouse":"/tmp/warehouse","spark.driver.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.sql.catalog.local":"org.apache.iceberg.spark.SparkCatalog","spark.submit.deployMode":"client","spark.master":"k8s://https://127.0.0.1:16443","spark.kubernetes.executor.podNamePrefix":"fresh-kube-bb0c5b8f311360be","spark.dynamicAllocation.enabled":"true","spark.sql.execution.id":"6","spark.sql.catalogImplementation":"hive","spark.kubernetes.authenticate.caCertFile":"/var/snap/microk8s/current/certs/ca.crt","spark.sql.catalog.spark_catalog.type":"hive","spark.app.id":"spark-a8d39974ec7b451a92b303fa953434e4"}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":202,"Index":68,"Attempt":0,"Partition ID":68,"Launch Time":1714515314507,"Executor ID":"1","Host":"10.1.246.124","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":203,"Index":0,"Attempt":0,"Partition ID":0,"Launch Time":1714515314542,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":202,"Index":68,"Attempt":0,"Partition ID":68,"Launch Time":1714515314507,"Executor ID":"1","Host":"10.1.246.124","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314543,"Failed":false,"Killed":false,"Accumulables":[{"ID":69,"Name":"local blocks read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":72,"Name":"local bytes read","Update":"84","Value":"84","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":73,"Name":"fetch wait time","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"records read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":5,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3057762,"Value":3057762,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":27,"Value":27,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":13766663,"Value":13766663,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":5114,"Value":5114,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":84,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":127,"Name":"internal.metrics.output.bytesWritten","Update":703,"Value":703,"Internal":true,"Count Failed Values":true},{"ID":128,"Name":"internal.metrics.output.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":3057762,"Executor Run Time":27,"Executor CPU Time":13766663,"Peak Execution Memory":0,"Result Size":5114,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":84,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":703,"Records Written":1},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":204,"Index":1,"Attempt":0,"Partition ID":1,"Launch Time":1714515314548,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":203,"Index":0,"Attempt":0,"Partition ID":0,"Launch Time":1714515314542,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314548,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1273005,"Value":4330767,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":28,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1219877,"Value":14986540,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":7627,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1273005,"Executor Run Time":1,"Executor CPU Time":1219877,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":205,"Index":2,"Attempt":0,"Partition ID":2,"Launch Time":1714515314552,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":204,"Index":1,"Attempt":0,"Partition ID":1,"Launch Time":1714515314548,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314552,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":7,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1265246,"Value":5596013,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":29,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1084232,"Value":16070772,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":10140,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1265246,"Executor Run Time":1,"Executor CPU Time":1084232,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":206,"Index":3,"Attempt":0,"Partition ID":3,"Launch Time":1714515314555,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":205,"Index":2,"Attempt":0,"Partition ID":2,"Launch Time":1714515314552,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314556,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":669176,"Value":6265189,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":30,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":842897,"Value":16913669,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":12610,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":669176,"Executor Run Time":1,"Executor CPU Time":842897,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":207,"Index":4,"Attempt":0,"Partition ID":4,"Launch Time":1714515314558,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":206,"Index":3,"Attempt":0,"Partition ID":3,"Launch Time":1714515314555,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314559,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":670166,"Value":6935355,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":31,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":733562,"Value":17647231,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":15080,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":670166,"Executor Run Time":1,"Executor CPU Time":733562,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":208,"Index":5,"Attempt":0,"Partition ID":5,"Launch Time":1714515314561,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":207,"Index":4,"Attempt":0,"Partition ID":4,"Launch Time":1714515314558,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314562,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":619738,"Value":7555093,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":665886,"Value":18313117,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":17507,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":619738,"Executor Run Time":0,"Executor CPU Time":665886,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":209,"Index":6,"Attempt":0,"Partition ID":6,"Launch Time":1714515314564,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":208,"Index":5,"Attempt":0,"Partition ID":5,"Launch Time":1714515314561,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314564,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":497965,"Value":8053058,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":565151,"Value":18878268,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":19934,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":497965,"Executor Run Time":0,"Executor CPU Time":565151,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":210,"Index":7,"Attempt":0,"Partition ID":7,"Launch Time":1714515314567,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":209,"Index":6,"Attempt":0,"Partition ID":6,"Launch Time":1714515314564,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314567,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":510715,"Value":8563773,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":587339,"Value":19465607,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":22361,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":510715,"Executor Run Time":0,"Executor CPU Time":587339,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":211,"Index":8,"Attempt":0,"Partition ID":8,"Launch Time":1714515314570,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":210,"Index":7,"Attempt":0,"Partition ID":7,"Launch Time":1714515314567,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314570,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":628228,"Value":9192001,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":634437,"Value":20100044,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":24788,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":628228,"Executor Run Time":0,"Executor CPU Time":634437,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":212,"Index":9,"Attempt":0,"Partition ID":9,"Launch Time":1714515314572,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":211,"Index":8,"Attempt":0,"Partition ID":8,"Launch Time":1714515314570,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314572,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":473256,"Value":9665257,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":622678,"Value":20722722,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":27215,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":473256,"Executor Run Time":0,"Executor CPU Time":622678,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":213,"Index":10,"Attempt":0,"Partition ID":10,"Launch Time":1714515314574,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":212,"Index":9,"Attempt":0,"Partition ID":9,"Launch Time":1714515314572,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314575,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":472726,"Value":10137983,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":524753,"Value":21247475,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":29642,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":472726,"Executor Run Time":0,"Executor CPU Time":524753,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":214,"Index":11,"Attempt":0,"Partition ID":11,"Launch Time":1714515314577,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":213,"Index":10,"Attempt":0,"Partition ID":10,"Launch Time":1714515314574,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314577,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":623089,"Value":10761072,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":647745,"Value":21895220,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":32069,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":623089,"Executor Run Time":0,"Executor CPU Time":647745,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":215,"Index":12,"Attempt":0,"Partition ID":12,"Launch Time":1714515314579,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":214,"Index":11,"Attempt":0,"Partition ID":11,"Launch Time":1714515314577,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314580,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":455457,"Value":11216529,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":574769,"Value":22469989,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":34496,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":455457,"Executor Run Time":0,"Executor CPU Time":574769,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":216,"Index":13,"Attempt":0,"Partition ID":13,"Launch Time":1714515314582,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":215,"Index":12,"Attempt":0,"Partition ID":12,"Launch Time":1714515314579,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314582,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":571331,"Value":11787860,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":655076,"Value":23125065,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":36923,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":571331,"Executor Run Time":0,"Executor CPU Time":655076,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":217,"Index":14,"Attempt":0,"Partition ID":14,"Launch Time":1714515314585,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":216,"Index":13,"Attempt":0,"Partition ID":13,"Launch Time":1714515314582,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314585,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":705304,"Value":12493164,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":668445,"Value":23793510,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":39350,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":705304,"Executor Run Time":0,"Executor CPU Time":668445,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":218,"Index":15,"Attempt":0,"Partition ID":15,"Launch Time":1714515314588,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":217,"Index":14,"Attempt":0,"Partition ID":14,"Launch Time":1714515314585,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314588,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":497944,"Value":12991108,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":574820,"Value":24368330,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":41777,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":497944,"Executor Run Time":0,"Executor CPU Time":574820,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":219,"Index":16,"Attempt":0,"Partition ID":16,"Launch Time":1714515314590,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":218,"Index":15,"Attempt":0,"Partition ID":15,"Launch Time":1714515314588,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314590,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":568661,"Value":13559769,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":611969,"Value":24980299,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":44204,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":568661,"Executor Run Time":0,"Executor CPU Time":611969,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":220,"Index":17,"Attempt":0,"Partition ID":17,"Launch Time":1714515314593,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":219,"Index":16,"Attempt":0,"Partition ID":16,"Launch Time":1714515314590,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314593,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":493175,"Value":14052944,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":554222,"Value":25534521,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":46631,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":493175,"Executor Run Time":0,"Executor CPU Time":554222,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":221,"Index":18,"Attempt":0,"Partition ID":18,"Launch Time":1714515314595,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":220,"Index":17,"Attempt":0,"Partition ID":17,"Launch Time":1714515314593,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314595,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":477005,"Value":14529949,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":546932,"Value":26081453,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":49058,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":477005,"Executor Run Time":0,"Executor CPU Time":546932,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":222,"Index":19,"Attempt":0,"Partition ID":19,"Launch Time":1714515314597,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":221,"Index":18,"Attempt":0,"Partition ID":18,"Launch Time":1714515314595,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314598,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":627968,"Value":15157917,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":554042,"Value":26635495,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":51485,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":627968,"Executor Run Time":0,"Executor CPU Time":554042,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":223,"Index":20,"Attempt":0,"Partition ID":20,"Launch Time":1714515314600,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":222,"Index":19,"Attempt":0,"Partition ID":19,"Launch Time":1714515314597,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314600,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":453757,"Value":15611674,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":528703,"Value":27164198,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":53912,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":453757,"Executor Run Time":0,"Executor CPU Time":528703,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":224,"Index":21,"Attempt":0,"Partition ID":21,"Launch Time":1714515314603,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":223,"Index":20,"Attempt":0,"Partition ID":20,"Launch Time":1714515314600,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314603,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":490824,"Value":16102498,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":582851,"Value":27747049,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":56339,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":490824,"Executor Run Time":0,"Executor CPU Time":582851,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":225,"Index":22,"Attempt":0,"Partition ID":22,"Launch Time":1714515314605,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":224,"Index":21,"Attempt":0,"Partition ID":21,"Launch Time":1714515314603,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314606,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":568911,"Value":16671409,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":679044,"Value":28426093,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":58766,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":568911,"Executor Run Time":0,"Executor CPU Time":679044,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":226,"Index":23,"Attempt":0,"Partition ID":23,"Launch Time":1714515314608,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":225,"Index":22,"Attempt":0,"Partition ID":22,"Launch Time":1714515314605,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314608,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":477126,"Value":17148535,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":636466,"Value":29062559,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":61193,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":477126,"Executor Run Time":0,"Executor CPU Time":636466,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":227,"Index":24,"Attempt":0,"Partition ID":24,"Launch Time":1714515314610,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":226,"Index":23,"Attempt":0,"Partition ID":23,"Launch Time":1714515314608,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314610,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":492044,"Value":17640579,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":572280,"Value":29634839,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":63620,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":492044,"Executor Run Time":0,"Executor CPU Time":572280,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":228,"Index":25,"Attempt":0,"Partition ID":25,"Launch Time":1714515314613,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":227,"Index":24,"Attempt":0,"Partition ID":24,"Launch Time":1714515314610,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314613,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":484145,"Value":18124724,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":600469,"Value":30235308,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":66047,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":484145,"Executor Run Time":0,"Executor CPU Time":600469,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":229,"Index":26,"Attempt":0,"Partition ID":26,"Launch Time":1714515314615,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":228,"Index":25,"Attempt":0,"Partition ID":25,"Launch Time":1714515314613,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314615,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":516194,"Value":18640918,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":624867,"Value":30860175,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":68474,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":516194,"Executor Run Time":0,"Executor CPU Time":624867,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":230,"Index":27,"Attempt":0,"Partition ID":27,"Launch Time":1714515314618,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":229,"Index":26,"Attempt":0,"Partition ID":26,"Launch Time":1714515314615,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314619,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":500044,"Value":19140962,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":32,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1147322,"Value":32007497,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":70944,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":500044,"Executor Run Time":1,"Executor CPU Time":1147322,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":231,"Index":28,"Attempt":0,"Partition ID":28,"Launch Time":1714515314621,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":230,"Index":27,"Attempt":0,"Partition ID":27,"Launch Time":1714515314618,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314622,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":611069,"Value":19752031,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":33,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":793319,"Value":32800816,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":73414,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":611069,"Executor Run Time":1,"Executor CPU Time":793319,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":232,"Index":29,"Attempt":0,"Partition ID":29,"Launch Time":1714515314625,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":231,"Index":28,"Attempt":0,"Partition ID":28,"Launch Time":1714515314621,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314625,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":636098,"Value":20388129,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":34,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":900843,"Value":33701659,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":75884,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":636098,"Executor Run Time":1,"Executor CPU Time":900843,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":233,"Index":30,"Attempt":0,"Partition ID":30,"Launch Time":1714515314628,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":232,"Index":29,"Attempt":0,"Partition ID":29,"Launch Time":1714515314625,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314628,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":544761,"Value":20932890,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":668876,"Value":34370535,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":78311,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":544761,"Executor Run Time":0,"Executor CPU Time":668876,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":234,"Index":31,"Attempt":0,"Partition ID":31,"Launch Time":1714515314631,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":233,"Index":30,"Attempt":0,"Partition ID":30,"Launch Time":1714515314628,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314631,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":646917,"Value":21579807,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":35,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":735472,"Value":35106007,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":80781,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":646917,"Executor Run Time":1,"Executor CPU Time":735472,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":235,"Index":32,"Attempt":0,"Partition ID":32,"Launch Time":1714515314635,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":234,"Index":31,"Attempt":0,"Partition ID":31,"Launch Time":1714515314631,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314635,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":811848,"Value":22391655,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":36,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1384229,"Value":36490236,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":83251,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":811848,"Executor Run Time":1,"Executor CPU Time":1384229,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":236,"Index":33,"Attempt":0,"Partition ID":33,"Launch Time":1714515314639,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":235,"Index":32,"Attempt":0,"Partition ID":32,"Launch Time":1714515314635,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314639,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":800998,"Value":23192653,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":37,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1082365,"Value":37572601,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":85721,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":800998,"Executor Run Time":1,"Executor CPU Time":1082365,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":237,"Index":34,"Attempt":0,"Partition ID":34,"Launch Time":1714515314642,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":236,"Index":33,"Attempt":0,"Partition ID":33,"Launch Time":1714515314639,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314643,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":688414,"Value":23881067,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":38,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":918054,"Value":38490655,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":88191,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":688414,"Executor Run Time":1,"Executor CPU Time":918054,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":238,"Index":35,"Attempt":0,"Partition ID":35,"Launch Time":1714515314645,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":237,"Index":34,"Attempt":0,"Partition ID":34,"Launch Time":1714515314642,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314646,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":690623,"Value":24571690,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":39,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":865046,"Value":39355701,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":90661,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":690623,"Executor Run Time":1,"Executor CPU Time":865046,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":239,"Index":36,"Attempt":0,"Partition ID":36,"Launch Time":1714515314649,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":238,"Index":35,"Attempt":0,"Partition ID":35,"Launch Time":1714515314645,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314649,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":627198,"Value":25198888,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":40,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":855275,"Value":40210976,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":93131,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":627198,"Executor Run Time":1,"Executor CPU Time":855275,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":240,"Index":37,"Attempt":0,"Partition ID":37,"Launch Time":1714515314653,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":239,"Index":36,"Attempt":0,"Partition ID":36,"Launch Time":1714515314649,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314653,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":899913,"Value":26098801,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":41,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1115123,"Value":41326099,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":95601,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":899913,"Executor Run Time":1,"Executor CPU Time":1115123,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":241,"Index":38,"Attempt":0,"Partition ID":38,"Launch Time":1714515314658,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":240,"Index":37,"Attempt":0,"Partition ID":37,"Launch Time":1714515314653,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314658,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":756891,"Value":26855692,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":42,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1172890,"Value":42498989,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":98071,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":756891,"Executor Run Time":1,"Executor CPU Time":1172890,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":242,"Index":39,"Attempt":0,"Partition ID":39,"Launch Time":1714515314662,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":241,"Index":38,"Attempt":0,"Partition ID":38,"Launch Time":1714515314658,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314662,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":816978,"Value":27672670,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":43,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1090434,"Value":43589423,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":100541,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":816978,"Executor Run Time":1,"Executor CPU Time":1090434,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":243,"Index":40,"Attempt":0,"Partition ID":40,"Launch Time":1714515314665,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":242,"Index":39,"Attempt":0,"Partition ID":39,"Launch Time":1714515314662,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314665,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":698004,"Value":28370674,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":44,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":856625,"Value":44446048,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":103011,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":698004,"Executor Run Time":1,"Executor CPU Time":856625,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":244,"Index":41,"Attempt":0,"Partition ID":41,"Launch Time":1714515314669,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":243,"Index":40,"Attempt":0,"Partition ID":40,"Launch Time":1714515314665,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314669,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":809408,"Value":29180082,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":45,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1121863,"Value":45567911,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":105481,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":809408,"Executor Run Time":1,"Executor CPU Time":1121863,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":245,"Index":42,"Attempt":0,"Partition ID":42,"Launch Time":1714515314672,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":244,"Index":41,"Attempt":0,"Partition ID":41,"Launch Time":1714515314669,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314673,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":660485,"Value":29840567,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":46,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":922703,"Value":46490614,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":107951,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":660485,"Executor Run Time":1,"Executor CPU Time":922703,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":246,"Index":43,"Attempt":0,"Partition ID":43,"Launch Time":1714515314676,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":245,"Index":42,"Attempt":0,"Partition ID":42,"Launch Time":1714515314672,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314676,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":767930,"Value":30608497,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":47,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":925323,"Value":47415937,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":110421,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":767930,"Executor Run Time":1,"Executor CPU Time":925323,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":247,"Index":44,"Attempt":0,"Partition ID":44,"Launch Time":1714515314679,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":246,"Index":43,"Attempt":0,"Partition ID":43,"Launch Time":1714515314676,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314679,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":633668,"Value":31242165,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":800938,"Value":48216875,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":112848,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":633668,"Executor Run Time":0,"Executor CPU Time":800938,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":248,"Index":45,"Attempt":0,"Partition ID":45,"Launch Time":1714515314682,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":247,"Index":44,"Attempt":0,"Partition ID":44,"Launch Time":1714515314679,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314682,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":668175,"Value":31910340,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":765751,"Value":48982626,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":115275,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":668175,"Executor Run Time":0,"Executor CPU Time":765751,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":249,"Index":46,"Attempt":0,"Partition ID":46,"Launch Time":1714515314685,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":248,"Index":45,"Attempt":0,"Partition ID":45,"Launch Time":1714515314682,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314685,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":654506,"Value":32564846,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":48,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1205698,"Value":50188324,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":117745,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":654506,"Executor Run Time":1,"Executor CPU Time":1205698,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":250,"Index":47,"Attempt":0,"Partition ID":47,"Launch Time":1714515314689,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":249,"Index":46,"Attempt":0,"Partition ID":46,"Launch Time":1714515314685,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314689,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":714993,"Value":33279839,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":49,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1067905,"Value":51256229,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":120215,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":714993,"Executor Run Time":1,"Executor CPU Time":1067905,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":251,"Index":48,"Attempt":0,"Partition ID":48,"Launch Time":1714515314692,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":250,"Index":47,"Attempt":0,"Partition ID":47,"Launch Time":1714515314689,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314692,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":637107,"Value":33916946,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":50,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":818079,"Value":52074308,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":122685,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":637107,"Executor Run Time":1,"Executor CPU Time":818079,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":252,"Index":49,"Attempt":0,"Partition ID":49,"Launch Time":1714515314695,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":251,"Index":48,"Attempt":0,"Partition ID":48,"Launch Time":1714515314692,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314695,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":655567,"Value":34572513,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":51,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":864815,"Value":52939123,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":125155,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":655567,"Executor Run Time":1,"Executor CPU Time":864815,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":253,"Index":50,"Attempt":0,"Partition ID":50,"Launch Time":1714515314698,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":252,"Index":49,"Attempt":0,"Partition ID":49,"Launch Time":1714515314695,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314699,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":881535,"Value":35454048,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":52,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":961790,"Value":53900913,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":127625,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":881535,"Executor Run Time":1,"Executor CPU Time":961790,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":254,"Index":51,"Attempt":0,"Partition ID":51,"Launch Time":1714515314702,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":253,"Index":50,"Attempt":0,"Partition ID":50,"Launch Time":1714515314698,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314703,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":770241,"Value":36224289,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":53,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1192308,"Value":55093221,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":130095,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":770241,"Executor Run Time":1,"Executor CPU Time":1192308,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":255,"Index":52,"Attempt":0,"Partition ID":52,"Launch Time":1714515314706,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":254,"Index":51,"Attempt":0,"Partition ID":51,"Launch Time":1714515314702,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314706,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":685295,"Value":36909584,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":54,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":940922,"Value":56034143,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":132565,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":685295,"Executor Run Time":1,"Executor CPU Time":940922,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":256,"Index":53,"Attempt":0,"Partition ID":53,"Launch Time":1714515314710,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":255,"Index":52,"Attempt":0,"Partition ID":52,"Launch Time":1714515314706,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314710,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":855445,"Value":37765029,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":55,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1143152,"Value":57177295,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":135035,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":855445,"Executor Run Time":1,"Executor CPU Time":1143152,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":257,"Index":54,"Attempt":0,"Partition ID":54,"Launch Time":1714515314714,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":256,"Index":53,"Attempt":0,"Partition ID":53,"Launch Time":1714515314710,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314714,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":963180,"Value":38728209,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":56,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1130762,"Value":58308057,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":137505,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":963180,"Executor Run Time":1,"Executor CPU Time":1130762,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":258,"Index":55,"Attempt":0,"Partition ID":55,"Launch Time":1714515314719,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":257,"Index":54,"Attempt":0,"Partition ID":54,"Launch Time":1714515314714,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314719,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":825027,"Value":39553236,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":2,"Value":58,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1684174,"Value":59992231,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":139975,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":825027,"Executor Run Time":2,"Executor CPU Time":1684174,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":259,"Index":56,"Attempt":0,"Partition ID":56,"Launch Time":1714515314724,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":258,"Index":55,"Attempt":0,"Partition ID":55,"Launch Time":1714515314719,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314724,"Failed":false,"Killed":false,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":8,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1143371,"Value":40696607,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":59,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1362570,"Value":61354801,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2513,"Value":142488,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1143371,"Executor Run Time":1,"Executor CPU Time":1362570,"Peak Execution Memory":0,"Result Size":2513,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":260,"Index":57,"Attempt":0,"Partition ID":57,"Launch Time":1714515314728,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":259,"Index":56,"Attempt":0,"Partition ID":56,"Launch Time":1714515314724,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314728,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":795369,"Value":41491976,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1027977,"Value":62382778,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":144958,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":795369,"Executor Run Time":1,"Executor CPU Time":1027977,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":261,"Index":58,"Attempt":0,"Partition ID":58,"Launch Time":1714515314731,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":260,"Index":57,"Attempt":0,"Partition ID":57,"Launch Time":1714515314728,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314731,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":659726,"Value":42151702,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":61,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":884404,"Value":63267182,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":147428,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":659726,"Executor Run Time":1,"Executor CPU Time":884404,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":262,"Index":59,"Attempt":0,"Partition ID":59,"Launch Time":1714515314734,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":261,"Index":58,"Attempt":0,"Partition ID":58,"Launch Time":1714515314731,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314734,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":606669,"Value":42758371,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":62,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1109203,"Value":64376385,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":149898,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":606669,"Executor Run Time":1,"Executor CPU Time":1109203,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":263,"Index":60,"Attempt":0,"Partition ID":60,"Launch Time":1714515314737,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":262,"Index":59,"Attempt":0,"Partition ID":59,"Launch Time":1714515314734,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314737,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":630788,"Value":43389159,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":796398,"Value":65172783,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":152325,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":630788,"Executor Run Time":0,"Executor CPU Time":796398,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":264,"Index":61,"Attempt":0,"Partition ID":61,"Launch Time":1714515314740,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":263,"Index":60,"Attempt":0,"Partition ID":60,"Launch Time":1714515314737,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314740,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":668786,"Value":44057945,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":813448,"Value":65986231,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":154752,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":668786,"Executor Run Time":0,"Executor CPU Time":813448,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":265,"Index":62,"Attempt":0,"Partition ID":62,"Launch Time":1714515314743,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":264,"Index":61,"Attempt":0,"Partition ID":61,"Launch Time":1714515314740,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314743,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":571130,"Value":44629075,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":775140,"Value":66761371,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":157179,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":571130,"Executor Run Time":0,"Executor CPU Time":775140,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":266,"Index":63,"Attempt":0,"Partition ID":63,"Launch Time":1714515314746,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":265,"Index":62,"Attempt":0,"Partition ID":62,"Launch Time":1714515314743,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314746,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":611019,"Value":45240094,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":850906,"Value":67612277,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":159649,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":611019,"Executor Run Time":1,"Executor CPU Time":850906,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":267,"Index":64,"Attempt":0,"Partition ID":64,"Launch Time":1714515314749,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":266,"Index":63,"Attempt":0,"Partition ID":63,"Launch Time":1714515314746,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314750,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":785109,"Value":46025203,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":64,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":988319,"Value":68600596,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":162119,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":785109,"Executor Run Time":1,"Executor CPU Time":988319,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":268,"Index":65,"Attempt":0,"Partition ID":65,"Launch Time":1714515314753,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":267,"Index":64,"Attempt":0,"Partition ID":64,"Launch Time":1714515314749,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314753,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":790269,"Value":46815472,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":65,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":991499,"Value":69592095,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":164589,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":790269,"Executor Run Time":1,"Executor CPU Time":991499,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":269,"Index":66,"Attempt":0,"Partition ID":66,"Launch Time":1714515314756,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":268,"Index":65,"Attempt":0,"Partition ID":65,"Launch Time":1714515314753,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314757,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":677676,"Value":47493148,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":66,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1158889,"Value":70750984,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":167059,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":677676,"Executor Run Time":1,"Executor CPU Time":1158889,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":270,"Index":67,"Attempt":0,"Partition ID":67,"Launch Time":1714515314760,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":269,"Index":66,"Attempt":0,"Partition ID":66,"Launch Time":1714515314756,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314760,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":710624,"Value":48203772,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":67,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":968919,"Value":71719903,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":169529,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":710624,"Executor Run Time":1,"Executor CPU Time":968919,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":271,"Index":69,"Attempt":0,"Partition ID":69,"Launch Time":1714515314763,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":270,"Index":67,"Attempt":0,"Partition ID":67,"Launch Time":1714515314760,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314763,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":845607,"Value":49049379,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":68,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1026897,"Value":72746800,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":171999,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":845607,"Executor Run Time":1,"Executor CPU Time":1026897,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":272,"Index":70,"Attempt":0,"Partition ID":70,"Launch Time":1714515314766,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":271,"Index":69,"Attempt":0,"Partition ID":69,"Launch Time":1714515314763,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314766,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":643777,"Value":49693156,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":69,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":874995,"Value":73621795,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":174469,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":643777,"Executor Run Time":1,"Executor CPU Time":874995,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":273,"Index":71,"Attempt":0,"Partition ID":71,"Launch Time":1714515314769,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":272,"Index":70,"Attempt":0,"Partition ID":70,"Launch Time":1714515314766,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314770,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":659085,"Value":50352241,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":70,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1046346,"Value":74668141,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":176939,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":659085,"Executor Run Time":1,"Executor CPU Time":1046346,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":274,"Index":72,"Attempt":0,"Partition ID":72,"Launch Time":1714515314773,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":273,"Index":71,"Attempt":0,"Partition ID":71,"Launch Time":1714515314769,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314773,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":651286,"Value":51003527,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":71,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":907964,"Value":75576105,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":179409,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":651286,"Executor Run Time":1,"Executor CPU Time":907964,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":275,"Index":73,"Attempt":0,"Partition ID":73,"Launch Time":1714515314775,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":274,"Index":72,"Attempt":0,"Partition ID":72,"Launch Time":1714515314773,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314776,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":600949,"Value":51604476,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":778710,"Value":76354815,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":181836,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":600949,"Executor Run Time":0,"Executor CPU Time":778710,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":276,"Index":74,"Attempt":0,"Partition ID":74,"Launch Time":1714515314778,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":275,"Index":73,"Attempt":0,"Partition ID":73,"Launch Time":1714515314775,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314779,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":733522,"Value":52337998,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":72,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":863386,"Value":77218201,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":184306,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":733522,"Executor Run Time":1,"Executor CPU Time":863386,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":277,"Index":75,"Attempt":0,"Partition ID":75,"Launch Time":1714515314781,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":276,"Index":74,"Attempt":0,"Partition ID":74,"Launch Time":1714515314778,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314781,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":566881,"Value":52904879,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":763341,"Value":77981542,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":186733,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":566881,"Executor Run Time":0,"Executor CPU Time":763341,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":278,"Index":76,"Attempt":0,"Partition ID":76,"Launch Time":1714515314784,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":277,"Index":75,"Attempt":0,"Partition ID":75,"Launch Time":1714515314781,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314784,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":674865,"Value":53579744,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":73,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":965660,"Value":78947202,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":189203,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":674865,"Executor Run Time":1,"Executor CPU Time":965660,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":279,"Index":77,"Attempt":0,"Partition ID":77,"Launch Time":1714515314787,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":278,"Index":76,"Attempt":0,"Partition ID":76,"Launch Time":1714515314784,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314787,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":530362,"Value":54110106,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":74,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":793130,"Value":79740332,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":191673,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":530362,"Executor Run Time":1,"Executor CPU Time":793130,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":280,"Index":78,"Attempt":0,"Partition ID":78,"Launch Time":1714515314790,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":279,"Index":77,"Attempt":0,"Partition ID":77,"Launch Time":1714515314787,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314790,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":611768,"Value":54721874,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":75,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":859477,"Value":80599809,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":194143,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":611768,"Executor Run Time":1,"Executor CPU Time":859477,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":281,"Index":79,"Attempt":0,"Partition ID":79,"Launch Time":1714515314793,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":280,"Index":78,"Attempt":0,"Partition ID":78,"Launch Time":1714515314790,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314793,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":623919,"Value":55345793,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":76,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":894993,"Value":81494802,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":196613,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":623919,"Executor Run Time":1,"Executor CPU Time":894993,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":282,"Index":80,"Attempt":0,"Partition ID":80,"Launch Time":1714515314796,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":281,"Index":79,"Attempt":0,"Partition ID":79,"Launch Time":1714515314793,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314796,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":688814,"Value":56034607,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":77,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":850767,"Value":82345569,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":199083,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":688814,"Executor Run Time":1,"Executor CPU Time":850767,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":283,"Index":81,"Attempt":0,"Partition ID":81,"Launch Time":1714515314798,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":282,"Index":80,"Attempt":0,"Partition ID":80,"Launch Time":1714515314796,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314798,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":496854,"Value":56531461,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":654796,"Value":83000365,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":201510,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":496854,"Executor Run Time":0,"Executor CPU Time":654796,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":284,"Index":82,"Attempt":0,"Partition ID":82,"Launch Time":1714515314801,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":283,"Index":81,"Attempt":0,"Partition ID":81,"Launch Time":1714515314798,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314801,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":538782,"Value":57070243,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":78,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":838057,"Value":83838422,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":203980,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":538782,"Executor Run Time":1,"Executor CPU Time":838057,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":285,"Index":83,"Attempt":0,"Partition ID":83,"Launch Time":1714515314803,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":284,"Index":82,"Attempt":0,"Partition ID":82,"Launch Time":1714515314801,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314803,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":512974,"Value":57583217,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":666604,"Value":84505026,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":206407,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":512974,"Executor Run Time":0,"Executor CPU Time":666604,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":286,"Index":84,"Attempt":0,"Partition ID":84,"Launch Time":1714515314805,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":285,"Index":83,"Attempt":0,"Partition ID":83,"Launch Time":1714515314803,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314806,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":459906,"Value":58043123,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":599209,"Value":85104235,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":208834,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":459906,"Executor Run Time":0,"Executor CPU Time":599209,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":287,"Index":85,"Attempt":0,"Partition ID":85,"Launch Time":1714515314808,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":286,"Index":84,"Attempt":0,"Partition ID":84,"Launch Time":1714515314805,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314808,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":611088,"Value":58654211,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":744952,"Value":85849187,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":211261,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":611088,"Executor Run Time":0,"Executor CPU Time":744952,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":288,"Index":86,"Attempt":0,"Partition ID":86,"Launch Time":1714515314810,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":287,"Index":85,"Attempt":0,"Partition ID":85,"Launch Time":1714515314808,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314811,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":501654,"Value":59155865,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":649686,"Value":86498873,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":213688,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":501654,"Executor Run Time":0,"Executor CPU Time":649686,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":289,"Index":87,"Attempt":0,"Partition ID":87,"Launch Time":1714515314813,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":288,"Index":86,"Attempt":0,"Partition ID":86,"Launch Time":1714515314810,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314813,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":508294,"Value":59664159,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":620308,"Value":87119181,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":216115,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":508294,"Executor Run Time":0,"Executor CPU Time":620308,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":290,"Index":88,"Attempt":0,"Partition ID":88,"Launch Time":1714515314815,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":289,"Index":87,"Attempt":0,"Partition ID":87,"Launch Time":1714515314813,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314815,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":500484,"Value":60164643,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":628638,"Value":87747819,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":218542,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":500484,"Executor Run Time":0,"Executor CPU Time":628638,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":291,"Index":89,"Attempt":0,"Partition ID":89,"Launch Time":1714515314818,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":290,"Index":88,"Attempt":0,"Partition ID":88,"Launch Time":1714515314815,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314818,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":642977,"Value":60807620,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":79,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":887374,"Value":88635193,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":221012,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":642977,"Executor Run Time":1,"Executor CPU Time":887374,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":292,"Index":90,"Attempt":0,"Partition ID":90,"Launch Time":1714515314820,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":291,"Index":89,"Attempt":0,"Partition ID":89,"Launch Time":1714515314818,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314821,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":508744,"Value":61316364,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":80,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":745572,"Value":89380765,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":223482,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":508744,"Executor Run Time":1,"Executor CPU Time":745572,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":293,"Index":91,"Attempt":0,"Partition ID":91,"Launch Time":1714515314823,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":292,"Index":90,"Attempt":0,"Partition ID":90,"Launch Time":1714515314820,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314823,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":602779,"Value":61919143,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":753721,"Value":90134486,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":225909,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":602779,"Executor Run Time":0,"Executor CPU Time":753721,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":294,"Index":92,"Attempt":0,"Partition ID":92,"Launch Time":1714515314826,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":293,"Index":91,"Attempt":0,"Partition ID":91,"Launch Time":1714515314823,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314826,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":499044,"Value":62418187,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":639247,"Value":90773733,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":228336,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":499044,"Executor Run Time":0,"Executor CPU Time":639247,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":295,"Index":93,"Attempt":0,"Partition ID":93,"Launch Time":1714515314828,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":294,"Index":92,"Attempt":0,"Partition ID":92,"Launch Time":1714515314826,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314828,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":507554,"Value":62925741,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":703175,"Value":91476908,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":230763,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":507554,"Executor Run Time":0,"Executor CPU Time":703175,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":296,"Index":94,"Attempt":0,"Partition ID":94,"Launch Time":1714515314831,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":295,"Index":93,"Attempt":0,"Partition ID":93,"Launch Time":1714515314828,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314831,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":493325,"Value":63419066,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":645126,"Value":92122034,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":233190,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":493325,"Executor Run Time":0,"Executor CPU Time":645126,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":297,"Index":95,"Attempt":0,"Partition ID":95,"Launch Time":1714515314833,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":296,"Index":94,"Attempt":0,"Partition ID":94,"Launch Time":1714515314831,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314834,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":523254,"Value":63942320,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":730231,"Value":92852265,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":235617,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":523254,"Executor Run Time":0,"Executor CPU Time":730231,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":298,"Index":96,"Attempt":0,"Partition ID":96,"Launch Time":1714515314836,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":297,"Index":95,"Attempt":0,"Partition ID":95,"Launch Time":1714515314833,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314837,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":615848,"Value":64558168,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":81,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":850706,"Value":93702971,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":238087,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":615848,"Executor Run Time":1,"Executor CPU Time":850706,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":299,"Index":97,"Attempt":0,"Partition ID":97,"Launch Time":1714515314839,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":298,"Index":96,"Attempt":0,"Partition ID":96,"Launch Time":1714515314836,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314839,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":603589,"Value":65161757,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":82,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":782710,"Value":94485681,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":240557,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":603589,"Executor Run Time":1,"Executor CPU Time":782710,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":300,"Index":98,"Attempt":0,"Partition ID":98,"Launch Time":1714515314842,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":299,"Index":97,"Attempt":0,"Partition ID":97,"Launch Time":1714515314839,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314842,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":564680,"Value":65726437,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":83,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":753472,"Value":95239153,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":243027,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":564680,"Executor Run Time":1,"Executor CPU Time":753472,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":301,"Index":99,"Attempt":0,"Partition ID":99,"Launch Time":1714515314844,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":300,"Index":98,"Attempt":0,"Partition ID":98,"Launch Time":1714515314842,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314845,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":502495,"Value":66228932,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":690994,"Value":95930147,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":245454,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":502495,"Executor Run Time":0,"Executor CPU Time":690994,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":302,"Index":100,"Attempt":0,"Partition ID":100,"Launch Time":1714515314847,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":301,"Index":99,"Attempt":0,"Partition ID":99,"Launch Time":1714515314844,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314847,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":518133,"Value":66747065,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":672265,"Value":96602412,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":247881,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":518133,"Executor Run Time":0,"Executor CPU Time":672265,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":303,"Index":101,"Attempt":0,"Partition ID":101,"Launch Time":1714515314849,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":302,"Index":100,"Attempt":0,"Partition ID":100,"Launch Time":1714515314847,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314850,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":495714,"Value":67242779,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":672086,"Value":97274498,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":250308,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":495714,"Executor Run Time":0,"Executor CPU Time":672086,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":304,"Index":102,"Attempt":0,"Partition ID":102,"Launch Time":1714515314852,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":303,"Index":101,"Attempt":0,"Partition ID":101,"Launch Time":1714515314849,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314852,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":568041,"Value":67810820,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":791108,"Value":98065606,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":252735,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":568041,"Executor Run Time":0,"Executor CPU Time":791108,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":305,"Index":103,"Attempt":0,"Partition ID":103,"Launch Time":1714515314855,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":304,"Index":102,"Attempt":0,"Partition ID":102,"Launch Time":1714515314852,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314855,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":582701,"Value":68393521,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":769229,"Value":98834835,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":255162,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":582701,"Executor Run Time":0,"Executor CPU Time":769229,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":306,"Index":104,"Attempt":0,"Partition ID":104,"Launch Time":1714515314858,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":305,"Index":103,"Attempt":0,"Partition ID":103,"Launch Time":1714515314855,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314858,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":566732,"Value":68960253,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":748550,"Value":99583385,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":257589,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":566732,"Executor Run Time":0,"Executor CPU Time":748550,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":307,"Index":105,"Attempt":0,"Partition ID":105,"Launch Time":1714515314861,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":306,"Index":104,"Attempt":0,"Partition ID":104,"Launch Time":1714515314858,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314861,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":530193,"Value":69490446,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":740253,"Value":100323638,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":260016,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":530193,"Executor Run Time":0,"Executor CPU Time":740253,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":308,"Index":106,"Attempt":0,"Partition ID":106,"Launch Time":1714515314863,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":307,"Index":105,"Attempt":0,"Partition ID":105,"Launch Time":1714515314861,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314863,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":529133,"Value":70019579,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":691375,"Value":101015013,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":262443,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":529133,"Executor Run Time":0,"Executor CPU Time":691375,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":309,"Index":107,"Attempt":0,"Partition ID":107,"Launch Time":1714515314866,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":308,"Index":106,"Attempt":0,"Partition ID":106,"Launch Time":1714515314863,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314866,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":600109,"Value":70619688,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":720384,"Value":101735397,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":264870,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":600109,"Executor Run Time":0,"Executor CPU Time":720384,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":310,"Index":108,"Attempt":0,"Partition ID":108,"Launch Time":1714515314869,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":309,"Index":107,"Attempt":0,"Partition ID":107,"Launch Time":1714515314866,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314869,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":639367,"Value":71259055,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":923152,"Value":102658549,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":267340,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":639367,"Executor Run Time":1,"Executor CPU Time":923152,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":311,"Index":109,"Attempt":0,"Partition ID":109,"Launch Time":1714515314871,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":310,"Index":108,"Attempt":0,"Partition ID":108,"Launch Time":1714515314869,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314871,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":530482,"Value":71789537,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":723024,"Value":103381573,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":269767,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":530482,"Executor Run Time":0,"Executor CPU Time":723024,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":312,"Index":110,"Attempt":0,"Partition ID":110,"Launch Time":1714515314874,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":311,"Index":109,"Attempt":0,"Partition ID":109,"Launch Time":1714515314871,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314874,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":502414,"Value":72291951,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":694094,"Value":104075667,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":272194,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":502414,"Executor Run Time":0,"Executor CPU Time":694094,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":313,"Index":111,"Attempt":0,"Partition ID":111,"Launch Time":1714515314876,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":312,"Index":110,"Attempt":0,"Partition ID":110,"Launch Time":1714515314874,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314876,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":500285,"Value":72792236,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":652016,"Value":104727683,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":274621,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":500285,"Executor Run Time":0,"Executor CPU Time":652016,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":314,"Index":112,"Attempt":0,"Partition ID":112,"Launch Time":1714515314878,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":313,"Index":111,"Attempt":0,"Partition ID":111,"Launch Time":1714515314876,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314879,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":503854,"Value":73296090,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":673605,"Value":105401288,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":277048,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":503854,"Executor Run Time":0,"Executor CPU Time":673605,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":315,"Index":113,"Attempt":0,"Partition ID":113,"Launch Time":1714515314881,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":314,"Index":112,"Attempt":0,"Partition ID":112,"Launch Time":1714515314878,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314881,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":528343,"Value":73824433,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":690304,"Value":106091592,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":279475,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":528343,"Executor Run Time":0,"Executor CPU Time":690304,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":316,"Index":114,"Attempt":0,"Partition ID":114,"Launch Time":1714515314884,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":315,"Index":113,"Attempt":0,"Partition ID":113,"Launch Time":1714515314881,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314884,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":573370,"Value":74397803,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":700534,"Value":106792126,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":281902,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":573370,"Executor Run Time":0,"Executor CPU Time":700534,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":317,"Index":115,"Attempt":0,"Partition ID":115,"Launch Time":1714515314886,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":316,"Index":114,"Attempt":0,"Partition ID":114,"Launch Time":1714515314884,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314886,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":535192,"Value":74932995,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":724293,"Value":107516419,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":284329,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":535192,"Executor Run Time":0,"Executor CPU Time":724293,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":318,"Index":116,"Attempt":0,"Partition ID":116,"Launch Time":1714515314889,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":317,"Index":115,"Attempt":0,"Partition ID":115,"Launch Time":1714515314886,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314890,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":729423,"Value":75662418,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":85,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":874875,"Value":108391294,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":286799,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":729423,"Executor Run Time":1,"Executor CPU Time":874875,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":319,"Index":117,"Attempt":0,"Partition ID":117,"Launch Time":1714515314892,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":318,"Index":116,"Attempt":0,"Partition ID":116,"Launch Time":1714515314889,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314892,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":556351,"Value":76218769,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":735973,"Value":109127267,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":289226,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":556351,"Executor Run Time":0,"Executor CPU Time":735973,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":320,"Index":118,"Attempt":0,"Partition ID":118,"Launch Time":1714515314895,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":319,"Index":117,"Attempt":0,"Partition ID":117,"Launch Time":1714515314892,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314895,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":623437,"Value":76842206,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":728223,"Value":109855490,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":291653,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":623437,"Executor Run Time":0,"Executor CPU Time":728223,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":321,"Index":119,"Attempt":0,"Partition ID":119,"Launch Time":1714515314897,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":320,"Index":118,"Attempt":0,"Partition ID":118,"Launch Time":1714515314895,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314897,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":580490,"Value":77422696,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":647727,"Value":110503217,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":294080,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":580490,"Executor Run Time":0,"Executor CPU Time":647727,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":322,"Index":120,"Attempt":0,"Partition ID":120,"Launch Time":1714515314899,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":321,"Index":119,"Attempt":0,"Partition ID":119,"Launch Time":1714515314897,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314899,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":537233,"Value":77959929,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":693265,"Value":111196482,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":296507,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":537233,"Executor Run Time":0,"Executor CPU Time":693265,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":323,"Index":121,"Attempt":0,"Partition ID":121,"Launch Time":1714515314902,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":322,"Index":120,"Attempt":0,"Partition ID":120,"Launch Time":1714515314899,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314902,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":560940,"Value":78520869,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":648878,"Value":111845360,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":298934,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":560940,"Executor Run Time":0,"Executor CPU Time":648878,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":324,"Index":122,"Attempt":0,"Partition ID":122,"Launch Time":1714515314904,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":323,"Index":121,"Attempt":0,"Partition ID":121,"Launch Time":1714515314902,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314904,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":476136,"Value":78997005,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":580540,"Value":112425900,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":301361,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":476136,"Executor Run Time":0,"Executor CPU Time":580540,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":325,"Index":123,"Attempt":0,"Partition ID":123,"Launch Time":1714515314906,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":324,"Index":122,"Attempt":0,"Partition ID":122,"Launch Time":1714515314904,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314906,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":445727,"Value":79442732,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":550062,"Value":112975962,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":303788,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":445727,"Executor Run Time":0,"Executor CPU Time":550062,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":326,"Index":124,"Attempt":0,"Partition ID":124,"Launch Time":1714515314908,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":325,"Index":123,"Attempt":0,"Partition ID":123,"Launch Time":1714515314906,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314908,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":575721,"Value":80018453,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":664285,"Value":113640247,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":306215,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":575721,"Executor Run Time":0,"Executor CPU Time":664285,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":327,"Index":125,"Attempt":0,"Partition ID":125,"Launch Time":1714515314911,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":326,"Index":124,"Attempt":0,"Partition ID":124,"Launch Time":1714515314908,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314911,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":577370,"Value":80595823,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":587391,"Value":114227638,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":308642,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":577370,"Executor Run Time":0,"Executor CPU Time":587391,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":328,"Index":126,"Attempt":0,"Partition ID":126,"Launch Time":1714515314913,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":327,"Index":125,"Attempt":0,"Partition ID":125,"Launch Time":1714515314911,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314913,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":448267,"Value":81044090,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":546342,"Value":114773980,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":311069,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":448267,"Executor Run Time":0,"Executor CPU Time":546342,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":329,"Index":127,"Attempt":0,"Partition ID":127,"Launch Time":1714515314915,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":328,"Index":126,"Attempt":0,"Partition ID":126,"Launch Time":1714515314913,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314915,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":441778,"Value":81485868,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":532853,"Value":115306833,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":313496,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":441778,"Executor Run Time":0,"Executor CPU Time":532853,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":330,"Index":128,"Attempt":0,"Partition ID":128,"Launch Time":1714515314918,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":329,"Index":127,"Attempt":0,"Partition ID":127,"Launch Time":1714515314915,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314918,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":475305,"Value":81961173,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":687115,"Value":115993948,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":315923,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":475305,"Executor Run Time":0,"Executor CPU Time":687115,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":331,"Index":129,"Attempt":0,"Partition ID":129,"Launch Time":1714515314920,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":330,"Index":128,"Attempt":0,"Partition ID":128,"Launch Time":1714515314918,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314920,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":462146,"Value":82423319,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":727063,"Value":116721011,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":318350,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":462146,"Executor Run Time":0,"Executor CPU Time":727063,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":332,"Index":130,"Attempt":0,"Partition ID":130,"Launch Time":1714515314923,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":331,"Index":129,"Attempt":0,"Partition ID":129,"Launch Time":1714515314920,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314923,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":649087,"Value":83072406,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":86,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":676965,"Value":117397976,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":320820,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":649087,"Executor Run Time":1,"Executor CPU Time":676965,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":333,"Index":131,"Attempt":0,"Partition ID":131,"Launch Time":1714515314926,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":332,"Index":130,"Attempt":0,"Partition ID":130,"Launch Time":1714515314923,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314926,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":491495,"Value":83563901,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":645557,"Value":118043533,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":323247,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":491495,"Executor Run Time":0,"Executor CPU Time":645557,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":334,"Index":132,"Attempt":0,"Partition ID":132,"Launch Time":1714515314929,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":333,"Index":131,"Attempt":0,"Partition ID":131,"Launch Time":1714515314926,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314929,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":625548,"Value":84189449,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":87,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":787459,"Value":118830992,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":325717,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":625548,"Executor Run Time":1,"Executor CPU Time":787459,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":335,"Index":133,"Attempt":0,"Partition ID":133,"Launch Time":1714515314931,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":334,"Index":132,"Attempt":0,"Partition ID":132,"Launch Time":1714515314929,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314931,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":504015,"Value":84693464,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":646766,"Value":119477758,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":328144,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":504015,"Executor Run Time":0,"Executor CPU Time":646766,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":336,"Index":134,"Attempt":0,"Partition ID":134,"Launch Time":1714515314934,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":335,"Index":133,"Attempt":0,"Partition ID":133,"Launch Time":1714515314931,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314934,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":521814,"Value":85215278,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":728432,"Value":120206190,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":330571,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":521814,"Executor Run Time":0,"Executor CPU Time":728432,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":337,"Index":135,"Attempt":0,"Partition ID":135,"Launch Time":1714515314937,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":336,"Index":134,"Attempt":0,"Partition ID":134,"Launch Time":1714515314934,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314938,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":812708,"Value":86027986,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":88,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":899285,"Value":121105475,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":333041,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":812708,"Executor Run Time":1,"Executor CPU Time":899285,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":338,"Index":136,"Attempt":0,"Partition ID":136,"Launch Time":1714515314940,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":337,"Index":135,"Attempt":0,"Partition ID":135,"Launch Time":1714515314937,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314941,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":694645,"Value":86722631,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":89,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":866685,"Value":121972160,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":335511,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":694645,"Executor Run Time":1,"Executor CPU Time":866685,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":339,"Index":137,"Attempt":0,"Partition ID":137,"Launch Time":1714515314943,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":338,"Index":136,"Attempt":0,"Partition ID":136,"Launch Time":1714515314940,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314944,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":701184,"Value":87423815,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":90,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":856025,"Value":122828185,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":337981,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":701184,"Executor Run Time":1,"Executor CPU Time":856025,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":340,"Index":138,"Attempt":0,"Partition ID":138,"Launch Time":1714515314947,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":339,"Index":137,"Attempt":0,"Partition ID":137,"Launch Time":1714515314943,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314947,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":703204,"Value":88127019,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":91,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":818917,"Value":123647102,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":340451,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":703204,"Executor Run Time":1,"Executor CPU Time":818917,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":341,"Index":139,"Attempt":0,"Partition ID":139,"Launch Time":1714515314949,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":340,"Index":138,"Attempt":0,"Partition ID":138,"Launch Time":1714515314947,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314949,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":537733,"Value":88664752,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":625537,"Value":124272639,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":342878,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":537733,"Executor Run Time":0,"Executor CPU Time":625537,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":342,"Index":140,"Attempt":0,"Partition ID":140,"Launch Time":1714515314952,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":341,"Index":139,"Attempt":0,"Partition ID":139,"Launch Time":1714515314949,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314953,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":814908,"Value":89479660,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":92,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":947952,"Value":125220591,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":345348,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":814908,"Executor Run Time":1,"Executor CPU Time":947952,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":343,"Index":141,"Attempt":0,"Partition ID":141,"Launch Time":1714515314956,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":342,"Index":140,"Attempt":0,"Partition ID":140,"Launch Time":1714515314952,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314956,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":760931,"Value":90240591,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":93,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":944721,"Value":126165312,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":347818,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":760931,"Executor Run Time":1,"Executor CPU Time":944721,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":344,"Index":142,"Attempt":0,"Partition ID":142,"Launch Time":1714515314959,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":343,"Index":141,"Attempt":0,"Partition ID":141,"Launch Time":1714515314956,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314959,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":692185,"Value":90932776,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":94,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":870694,"Value":127036006,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":350288,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":692185,"Executor Run Time":1,"Executor CPU Time":870694,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":345,"Index":143,"Attempt":0,"Partition ID":143,"Launch Time":1714515314962,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":344,"Index":142,"Attempt":0,"Partition ID":142,"Launch Time":1714515314959,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314962,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":774900,"Value":91707676,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":975641,"Value":128011647,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":352758,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":774900,"Executor Run Time":1,"Executor CPU Time":975641,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":346,"Index":144,"Attempt":0,"Partition ID":144,"Launch Time":1714515314965,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":345,"Index":143,"Attempt":0,"Partition ID":143,"Launch Time":1714515314962,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314965,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":716643,"Value":92424319,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":96,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":846777,"Value":128858424,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":355228,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":716643,"Executor Run Time":1,"Executor CPU Time":846777,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":347,"Index":145,"Attempt":0,"Partition ID":145,"Launch Time":1714515314968,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":346,"Index":144,"Attempt":0,"Partition ID":144,"Launch Time":1714515314965,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314968,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":698694,"Value":93123013,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":97,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":980709,"Value":129839133,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":357698,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":698694,"Executor Run Time":1,"Executor CPU Time":980709,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":348,"Index":146,"Attempt":0,"Partition ID":146,"Launch Time":1714515314971,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":347,"Index":145,"Attempt":0,"Partition ID":145,"Launch Time":1714515314968,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314971,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":743321,"Value":93866334,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":98,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":937452,"Value":130776585,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":360168,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":743321,"Executor Run Time":1,"Executor CPU Time":937452,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":349,"Index":147,"Attempt":0,"Partition ID":147,"Launch Time":1714515314974,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":348,"Index":146,"Attempt":0,"Partition ID":146,"Launch Time":1714515314971,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314974,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":683736,"Value":94550070,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":841146,"Value":131617731,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":362595,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":683736,"Executor Run Time":0,"Executor CPU Time":841146,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":350,"Index":148,"Attempt":0,"Partition ID":148,"Launch Time":1714515314977,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":349,"Index":147,"Attempt":0,"Partition ID":147,"Launch Time":1714515314974,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314977,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":760981,"Value":95311051,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":817698,"Value":132435429,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":365022,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":760981,"Executor Run Time":0,"Executor CPU Time":817698,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":351,"Index":149,"Attempt":0,"Partition ID":149,"Launch Time":1714515314979,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":350,"Index":148,"Attempt":0,"Partition ID":148,"Launch Time":1714515314977,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314979,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":510345,"Value":95821396,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":672744,"Value":133108173,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":367449,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":510345,"Executor Run Time":0,"Executor CPU Time":672744,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":352,"Index":150,"Attempt":0,"Partition ID":150,"Launch Time":1714515314981,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":351,"Index":149,"Attempt":0,"Partition ID":149,"Launch Time":1714515314979,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314982,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":471327,"Value":96292723,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":635797,"Value":133743970,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":369876,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":471327,"Executor Run Time":0,"Executor CPU Time":635797,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":353,"Index":151,"Attempt":0,"Partition ID":151,"Launch Time":1714515314984,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":352,"Index":150,"Attempt":0,"Partition ID":150,"Launch Time":1714515314981,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314984,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":480086,"Value":96772809,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":99,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":827307,"Value":134571277,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":372346,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":480086,"Executor Run Time":1,"Executor CPU Time":827307,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":354,"Index":152,"Attempt":0,"Partition ID":152,"Launch Time":1714515314986,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":353,"Index":151,"Attempt":0,"Partition ID":151,"Launch Time":1714515314984,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314986,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":471106,"Value":97243915,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":604799,"Value":135176076,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":374773,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":471106,"Executor Run Time":0,"Executor CPU Time":604799,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":355,"Index":153,"Attempt":0,"Partition ID":153,"Launch Time":1714515314989,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":354,"Index":152,"Attempt":0,"Partition ID":152,"Launch Time":1714515314986,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314989,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":663456,"Value":97907371,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":768830,"Value":135944906,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":377200,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":663456,"Executor Run Time":0,"Executor CPU Time":768830,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":356,"Index":154,"Attempt":0,"Partition ID":154,"Launch Time":1714515314992,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":355,"Index":153,"Attempt":0,"Partition ID":153,"Launch Time":1714515314989,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314992,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":498494,"Value":98405865,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":697965,"Value":136642871,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":379627,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":498494,"Executor Run Time":0,"Executor CPU Time":697965,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":357,"Index":155,"Attempt":0,"Partition ID":155,"Launch Time":1714515314995,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":356,"Index":154,"Attempt":0,"Partition ID":154,"Launch Time":1714515314992,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314995,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":738372,"Value":99144237,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":100,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":947631,"Value":137590502,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":382097,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":738372,"Executor Run Time":1,"Executor CPU Time":947631,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":358,"Index":156,"Attempt":0,"Partition ID":156,"Launch Time":1714515314998,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":357,"Index":155,"Attempt":0,"Partition ID":155,"Launch Time":1714515314995,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515314998,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":523183,"Value":99667420,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":713314,"Value":138303816,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":384524,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":523183,"Executor Run Time":0,"Executor CPU Time":713314,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":359,"Index":157,"Attempt":0,"Partition ID":157,"Launch Time":1714515315001,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":358,"Index":156,"Attempt":0,"Partition ID":156,"Launch Time":1714515314998,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315001,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":569022,"Value":100236442,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":101,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":840645,"Value":139144461,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":386994,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":569022,"Executor Run Time":1,"Executor CPU Time":840645,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":360,"Index":158,"Attempt":0,"Partition ID":158,"Launch Time":1714515315004,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":359,"Index":157,"Attempt":0,"Partition ID":157,"Launch Time":1714515315001,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315004,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":747222,"Value":100983664,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":102,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":933922,"Value":140078383,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":389464,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":747222,"Executor Run Time":1,"Executor CPU Time":933922,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":361,"Index":159,"Attempt":0,"Partition ID":159,"Launch Time":1714515315007,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":360,"Index":158,"Attempt":0,"Partition ID":158,"Launch Time":1714515315004,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315007,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":718423,"Value":101702087,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":103,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":946522,"Value":141024905,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":391934,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":718423,"Executor Run Time":1,"Executor CPU Time":946522,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":362,"Index":160,"Attempt":0,"Partition ID":160,"Launch Time":1714515315010,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":361,"Index":159,"Attempt":0,"Partition ID":159,"Launch Time":1714515315007,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315010,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":588230,"Value":102290317,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":104,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":858726,"Value":141883631,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":394404,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":588230,"Executor Run Time":1,"Executor CPU Time":858726,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":363,"Index":161,"Attempt":0,"Partition ID":161,"Launch Time":1714515315014,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":362,"Index":160,"Attempt":0,"Partition ID":160,"Launch Time":1714515315010,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315014,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":846746,"Value":103137063,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":105,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1067465,"Value":142951096,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":396874,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":846746,"Executor Run Time":1,"Executor CPU Time":1067465,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":364,"Index":162,"Attempt":0,"Partition ID":162,"Launch Time":1714515315017,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":363,"Index":161,"Attempt":0,"Partition ID":161,"Launch Time":1714515315014,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315017,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":747822,"Value":103884885,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":106,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":972439,"Value":143923535,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":399344,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":747822,"Executor Run Time":1,"Executor CPU Time":972439,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":365,"Index":163,"Attempt":0,"Partition ID":163,"Launch Time":1714515315021,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":364,"Index":162,"Attempt":0,"Partition ID":162,"Launch Time":1714515315017,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315021,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":921743,"Value":104806628,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":107,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1040887,"Value":144964422,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":401814,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":921743,"Executor Run Time":1,"Executor CPU Time":1040887,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":366,"Index":164,"Attempt":0,"Partition ID":164,"Launch Time":1714515315024,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":365,"Index":163,"Attempt":0,"Partition ID":163,"Launch Time":1714515315021,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315024,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":868335,"Value":105674963,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":108,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":973559,"Value":145937981,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":404284,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":868335,"Executor Run Time":1,"Executor CPU Time":973559,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":367,"Index":165,"Attempt":0,"Partition ID":165,"Launch Time":1714515315027,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":366,"Index":164,"Attempt":0,"Partition ID":164,"Launch Time":1714515315024,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315028,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":784810,"Value":106459773,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":109,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":866256,"Value":146804237,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":406754,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":784810,"Executor Run Time":1,"Executor CPU Time":866256,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":368,"Index":166,"Attempt":0,"Partition ID":166,"Launch Time":1714515315031,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":367,"Index":165,"Attempt":0,"Partition ID":165,"Launch Time":1714515315027,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315031,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":778211,"Value":107237984,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":110,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":893813,"Value":147698050,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":409224,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":778211,"Executor Run Time":1,"Executor CPU Time":893813,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":369,"Index":167,"Attempt":0,"Partition ID":167,"Launch Time":1714515315034,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":368,"Index":166,"Attempt":0,"Partition ID":166,"Launch Time":1714515315031,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315035,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":869856,"Value":108107840,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":111,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":979820,"Value":148677870,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":411694,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":869856,"Executor Run Time":1,"Executor CPU Time":979820,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":370,"Index":168,"Attempt":0,"Partition ID":168,"Launch Time":1714515315038,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":369,"Index":167,"Attempt":0,"Partition ID":167,"Launch Time":1714515315034,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315038,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":987979,"Value":109095819,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":112,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1092983,"Value":149770853,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":414164,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":987979,"Executor Run Time":1,"Executor CPU Time":1092983,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":371,"Index":169,"Attempt":0,"Partition ID":169,"Launch Time":1714515315041,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":370,"Index":168,"Attempt":0,"Partition ID":168,"Launch Time":1714515315038,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315042,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":764061,"Value":109859880,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":113,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":895383,"Value":150666236,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":416634,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":764061,"Executor Run Time":1,"Executor CPU Time":895383,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":372,"Index":170,"Attempt":0,"Partition ID":170,"Launch Time":1714515315045,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":371,"Index":169,"Attempt":0,"Partition ID":169,"Launch Time":1714515315041,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315045,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":805219,"Value":110665099,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":114,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":894145,"Value":151560381,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":419104,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":805219,"Executor Run Time":1,"Executor CPU Time":894145,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":373,"Index":171,"Attempt":0,"Partition ID":171,"Launch Time":1714515315048,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":372,"Index":170,"Attempt":0,"Partition ID":170,"Launch Time":1714515315045,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315048,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":671105,"Value":111336204,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":770481,"Value":152330862,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":421531,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":671105,"Executor Run Time":0,"Executor CPU Time":770481,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":374,"Index":172,"Attempt":0,"Partition ID":172,"Launch Time":1714515315051,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":373,"Index":171,"Attempt":0,"Partition ID":171,"Launch Time":1714515315048,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315052,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":666915,"Value":112003119,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":115,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1181440,"Value":153512302,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":424001,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":666915,"Executor Run Time":1,"Executor CPU Time":1181440,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":375,"Index":173,"Attempt":0,"Partition ID":173,"Launch Time":1714515315054,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":374,"Index":172,"Attempt":0,"Partition ID":172,"Launch Time":1714515315051,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315054,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":605049,"Value":112608168,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":757230,"Value":154269532,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":426428,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":605049,"Executor Run Time":0,"Executor CPU Time":757230,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":376,"Index":174,"Attempt":0,"Partition ID":174,"Launch Time":1714515315057,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":375,"Index":173,"Attempt":0,"Partition ID":173,"Launch Time":1714515315054,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315057,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":576060,"Value":113184228,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":672225,"Value":154941757,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":428855,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":576060,"Executor Run Time":0,"Executor CPU Time":672225,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":377,"Index":175,"Attempt":0,"Partition ID":175,"Launch Time":1714515315059,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":376,"Index":174,"Attempt":0,"Partition ID":174,"Launch Time":1714515315057,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315059,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":530193,"Value":113714421,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":655906,"Value":155597663,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":431282,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":530193,"Executor Run Time":0,"Executor CPU Time":655906,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":378,"Index":176,"Attempt":0,"Partition ID":176,"Launch Time":1714515315062,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":377,"Index":175,"Attempt":0,"Partition ID":175,"Launch Time":1714515315059,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315062,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":656696,"Value":114371117,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":695004,"Value":156292667,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":433709,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":656696,"Executor Run Time":0,"Executor CPU Time":695004,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":379,"Index":177,"Attempt":0,"Partition ID":177,"Launch Time":1714515315064,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":378,"Index":176,"Attempt":0,"Partition ID":176,"Launch Time":1714515315062,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315064,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":512364,"Value":114883481,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":576390,"Value":156869057,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":436136,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":512364,"Executor Run Time":0,"Executor CPU Time":576390,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":380,"Index":178,"Attempt":0,"Partition ID":178,"Launch Time":1714515315066,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":379,"Index":177,"Attempt":0,"Partition ID":177,"Launch Time":1714515315064,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315066,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":431367,"Value":115314848,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":530383,"Value":157399440,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":438563,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":431367,"Executor Run Time":0,"Executor CPU Time":530383,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":381,"Index":179,"Attempt":0,"Partition ID":179,"Launch Time":1714515315068,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":380,"Index":178,"Attempt":0,"Partition ID":178,"Launch Time":1714515315066,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315069,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":595070,"Value":115909918,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":763940,"Value":158163380,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":440990,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":595070,"Executor Run Time":0,"Executor CPU Time":763940,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":382,"Index":180,"Attempt":0,"Partition ID":180,"Launch Time":1714515315077,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":381,"Index":179,"Attempt":0,"Partition ID":179,"Launch Time":1714515315068,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315077,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":652087,"Value":116562005,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":654885,"Value":158818265,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":443417,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":652087,"Executor Run Time":0,"Executor CPU Time":654885,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":383,"Index":181,"Attempt":0,"Partition ID":181,"Launch Time":1714515315079,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":382,"Index":180,"Attempt":0,"Partition ID":180,"Launch Time":1714515315077,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315079,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":427108,"Value":116989113,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":557251,"Value":159375516,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":445844,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":427108,"Executor Run Time":0,"Executor CPU Time":557251,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":384,"Index":182,"Attempt":0,"Partition ID":182,"Launch Time":1714515315081,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":383,"Index":181,"Attempt":0,"Partition ID":181,"Launch Time":1714515315079,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315082,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":447276,"Value":117436389,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":627338,"Value":160002854,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":448271,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":447276,"Executor Run Time":0,"Executor CPU Time":627338,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":385,"Index":183,"Attempt":0,"Partition ID":183,"Launch Time":1714515315085,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":384,"Index":182,"Attempt":0,"Partition ID":182,"Launch Time":1714515315081,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315085,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":496284,"Value":117932673,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":116,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":781020,"Value":160783874,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":450741,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":496284,"Executor Run Time":1,"Executor CPU Time":781020,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":386,"Index":184,"Attempt":0,"Partition ID":184,"Launch Time":1714515315088,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":385,"Index":183,"Attempt":0,"Partition ID":183,"Launch Time":1714515315085,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315088,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":592279,"Value":118524952,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":117,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":838618,"Value":161622492,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":453211,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":592279,"Executor Run Time":1,"Executor CPU Time":838618,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":387,"Index":185,"Attempt":0,"Partition ID":185,"Launch Time":1714515315091,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":386,"Index":184,"Attempt":0,"Partition ID":184,"Launch Time":1714515315088,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315091,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":563681,"Value":119088633,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":734242,"Value":162356734,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":455638,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":563681,"Executor Run Time":0,"Executor CPU Time":734242,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":388,"Index":186,"Attempt":0,"Partition ID":186,"Launch Time":1714515315093,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":387,"Index":185,"Attempt":0,"Partition ID":185,"Launch Time":1714515315091,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315093,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":483335,"Value":119571968,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":626289,"Value":162983023,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":458065,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":483335,"Executor Run Time":0,"Executor CPU Time":626289,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":389,"Index":187,"Attempt":0,"Partition ID":187,"Launch Time":1714515315096,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":388,"Index":186,"Attempt":0,"Partition ID":186,"Launch Time":1714515315093,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315097,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":433378,"Value":120005346,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":118,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":611549,"Value":163594572,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":460535,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":433378,"Executor Run Time":1,"Executor CPU Time":611549,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":390,"Index":188,"Attempt":0,"Partition ID":188,"Launch Time":1714515315098,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":389,"Index":187,"Attempt":0,"Partition ID":187,"Launch Time":1714515315096,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315099,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":425029,"Value":120430375,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":575249,"Value":164169821,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":462962,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":425029,"Executor Run Time":0,"Executor CPU Time":575249,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":391,"Index":189,"Attempt":0,"Partition ID":189,"Launch Time":1714515315101,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":390,"Index":188,"Attempt":0,"Partition ID":188,"Launch Time":1714515315098,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315101,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":462337,"Value":120892712,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":591289,"Value":164761110,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":465389,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":462337,"Executor Run Time":0,"Executor CPU Time":591289,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":392,"Index":190,"Attempt":0,"Partition ID":190,"Launch Time":1714515315103,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":391,"Index":189,"Attempt":0,"Partition ID":189,"Launch Time":1714515315101,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315103,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":469007,"Value":121361719,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":636476,"Value":165397586,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":467816,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":469007,"Executor Run Time":0,"Executor CPU Time":636476,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":393,"Index":191,"Attempt":0,"Partition ID":191,"Launch Time":1714515315105,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":392,"Index":190,"Attempt":0,"Partition ID":190,"Launch Time":1714515315103,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315106,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":394860,"Value":121756579,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":544532,"Value":165942118,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":470243,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":394860,"Executor Run Time":0,"Executor CPU Time":544532,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":394,"Index":192,"Attempt":0,"Partition ID":192,"Launch Time":1714515315117,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":393,"Index":191,"Attempt":0,"Partition ID":191,"Launch Time":1714515315105,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315118,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":708513,"Value":122465092,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":119,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1005538,"Value":166947656,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":472713,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":708513,"Executor Run Time":1,"Executor CPU Time":1005538,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":395,"Index":193,"Attempt":0,"Partition ID":193,"Launch Time":1714515315121,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":394,"Index":192,"Attempt":0,"Partition ID":192,"Launch Time":1714515315117,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315121,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":769000,"Value":123234092,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":120,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":1142582,"Value":168090238,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":475183,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":769000,"Executor Run Time":1,"Executor CPU Time":1142582,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":396,"Index":194,"Attempt":0,"Partition ID":194,"Launch Time":1714515315124,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":395,"Index":193,"Attempt":0,"Partition ID":193,"Launch Time":1714515315121,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315124,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":571101,"Value":123805193,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":121,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":855815,"Value":168946053,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":477653,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":571101,"Executor Run Time":1,"Executor CPU Time":855815,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":397,"Index":195,"Attempt":0,"Partition ID":195,"Launch Time":1714515315127,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":396,"Index":194,"Attempt":0,"Partition ID":194,"Launch Time":1714515315124,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315127,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":482346,"Value":124287539,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":765760,"Value":169711813,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2427,"Value":480080,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":482346,"Executor Run Time":0,"Executor CPU Time":765760,"Peak Execution Memory":0,"Result Size":2427,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":398,"Index":196,"Attempt":0,"Partition ID":196,"Launch Time":1714515315130,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":397,"Index":195,"Attempt":0,"Partition ID":195,"Launch Time":1714515315127,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315130,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":477745,"Value":124765284,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":122,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":787539,"Value":170499352,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":482550,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":477745,"Executor Run Time":1,"Executor CPU Time":787539,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":399,"Index":197,"Attempt":0,"Partition ID":197,"Launch Time":1714515315133,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":398,"Index":196,"Attempt":0,"Partition ID":196,"Launch Time":1714515315130,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315133,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":690355,"Value":125455639,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":123,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":916792,"Value":171416144,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":485020,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":690355,"Executor Run Time":1,"Executor CPU Time":916792,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":400,"Index":198,"Attempt":0,"Partition ID":198,"Launch Time":1714515315136,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":399,"Index":197,"Attempt":0,"Partition ID":197,"Launch Time":1714515315133,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315137,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":717733,"Value":126173372,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":124,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":979360,"Value":172395504,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":487490,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":717733,"Executor Run Time":1,"Executor CPU Time":979360,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":401,"Index":199,"Attempt":0,"Partition ID":199,"Launch Time":1714515315139,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":400,"Index":198,"Attempt":0,"Partition ID":198,"Launch Time":1714515315136,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315139,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":526183,"Value":126699555,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":125,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":797279,"Value":173192783,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":489960,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":526183,"Executor Run Time":1,"Executor CPU Time":797279,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":401,"Index":199,"Attempt":0,"Partition ID":199,"Launch Time":1714515315139,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315142,"Failed":false,"Killed":false,"Accumulables":[{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Update":590039,"Value":127289594,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Update":1,"Value":126,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Update":903235,"Value":174096018,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Update":2470,"Value":492430,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":590039,"Executor Run Time":1,"Executor CPU Time":903235,"Peak Execution Memory":0,"Result Size":2470,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"processLine at CliDriver.java:336","Number of Tasks":200,"RDD Info":[{"RDD ID":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"13\",\"name\":\"Exchange\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"UNORDERED","Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[4],"Details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1714515314503,"Completion Time":1714515315142,"Accumulables":[{"ID":69,"Name":"local blocks read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":72,"Name":"local bytes read","Value":"84","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":73,"Name":"fetch wait time","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"records read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":104,"Name":"internal.metrics.executorDeserializeTime","Value":8,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorDeserializeCpuTime","Value":127289594,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorRunTime","Value":126,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.executorCpuTime","Value":174096018,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.resultSize","Value":492430,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":119,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":84,"Internal":true,"Count Failed Values":true},{"ID":120,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.read.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":127,"Name":"internal.metrics.output.bytesWritten","Value":703,"Internal":true,"Count Failed Values":true},{"ID":128,"Name":"internal.metrics.output.recordsWritten","Value":1,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":3,"Completion Time":1714515315143,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":6,"time":1714515315206} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":7,"description":"\nINSERT INTO local.projects VALUES(\"krisnova\", \"aurae\")","details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nCommandResult (1)\n +- AppendData (6)\n +- AdaptiveSparkPlan (5)\n +- == Final Plan ==\n ShuffleQueryStage (4), Statistics(sizeInBytes=40.0 B, rowCount=1)\n +- Exchange (3)\n +- LocalTableScan (2)\n +- == Initial Plan ==\n Exchange (3)\n +- LocalTableScan (2)\n\n\n(1) CommandResult\nOutput: []\nArguments: \n\n(2) LocalTableScan\nOutput [2]: [creator#13, projectname#14]\nArguments: [creator#13, projectname#14]\n\n(3) Exchange\nInput [2]: [creator#13, projectname#14]\nArguments: hashpartitioning(creator#13, 200), REPARTITION_BY_NUM, [plan_id=42]\n\n(4) ShuffleQueryStage\nOutput [2]: [creator#13, projectname#14]\nArguments: 0\n\n(5) AdaptiveSparkPlan\nOutput [2]: [creator#13, projectname#14]\nArguments: isFinalPlan=true\n\n(6) AppendData\nInput [2]: [creator#13, projectname#14]\nArguments: org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$2453/0x00007505ed0efbf0@3b3d7eb6, IcebergWrite(table=local.projects, format=PARQUET)\n\n","sparkPlanInfo":{"nodeName":"CommandResult","simpleString":"CommandResult ","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":129,"metricType":"sum"}]},"time":1714515315211,"modifiedConfigs":{}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":7,"time":1714515315211} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":8,"description":"\nSELECT * FROM local.developers INNER JOIN local.projects ON local.projects.creator = local.developers.username","details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nAdaptiveSparkPlan (5)\n+- BroadcastHashJoin Inner BuildRight (4)\n :- BatchScan (1)\n +- BroadcastExchange (3)\n +- BatchScan (2)\n\n\n(1) BatchScan\nOutput [3]: [username#15, firstname#16, lastname#17]\nlocal.developers (branch=null) [filters=username IS NOT NULL, groupedBy=]\n\n(2) BatchScan\nOutput [2]: [creator#18, projectname#19]\nlocal.projects (branch=null) [filters=creator IS NOT NULL, groupedBy=]\n\n(3) BroadcastExchange\nInput [2]: [creator#18, projectname#19]\nArguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=66]\n\n(4) BroadcastHashJoin\nLeft keys [1]: [username#15]\nRight keys [1]: [creator#18]\nJoin condition: None\n\n(5) AdaptiveSparkPlan\nOutput [5]: [username#15, firstname#16, lastname#17, creator#18, projectname#19]\nArguments: isFinalPlan=false\n\n","sparkPlanInfo":{"nodeName":"AdaptiveSparkPlan","simpleString":"AdaptiveSparkPlan isFinalPlan=false","children":[{"nodeName":"BroadcastHashJoin","simpleString":"BroadcastHashJoin [username#15], [creator#18], Inner, BuildRight, false","children":[{"nodeName":"BatchScan","simpleString":"BatchScan[username#15, firstname#16, lastname#17] local.developers (branch=null) [filters=username IS NOT NULL, groupedBy=] RuntimeFilters: []","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":131,"metricType":"sum"},{"name":"number of file splits read","accumulatorId":132,"metricType":"v2Custom_org.apache.iceberg.spark.source.metrics.NumSplits"},{"name":"number of row deletes applied","accumulatorId":133,"metricType":"v2Custom_org.apache.iceberg.spark.source.metrics.NumDeletes"}]},{"nodeName":"BroadcastExchange","simpleString":"BroadcastExchange HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=66]","children":[{"nodeName":"BatchScan","simpleString":"BatchScan[creator#18, projectname#19] local.projects (branch=null) [filters=creator IS NOT NULL, groupedBy=] RuntimeFilters: []","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":139,"metricType":"sum"},{"name":"number of file splits read","accumulatorId":140,"metricType":"v2Custom_org.apache.iceberg.spark.source.metrics.NumSplits"},{"name":"number of row deletes applied","accumulatorId":141,"metricType":"v2Custom_org.apache.iceberg.spark.source.metrics.NumDeletes"}]}],"metadata":{},"metrics":[{"name":"time to broadcast","accumulatorId":138,"metricType":"timing"},{"name":"time to build","accumulatorId":137,"metricType":"timing"},{"name":"time to collect","accumulatorId":136,"metricType":"timing"},{"name":"number of output rows","accumulatorId":135,"metricType":"sum"},{"name":"data size","accumulatorId":134,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":130,"metricType":"sum"}]}],"metadata":{},"metrics":[]},"time":1714515315393,"modifiedConfigs":{}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate","executionId":8,"physicalPlanDescription":"== Physical Plan ==\nAdaptiveSparkPlan (9)\n+- == Current Plan ==\n BroadcastHashJoin Inner BuildRight (6)\n :- BatchScan (1)\n +- BroadcastQueryStage (5)\n +- BroadcastExchange (4)\n +- * ColumnarToRow (3)\n +- BatchScan (2)\n+- == Initial Plan ==\n BroadcastHashJoin Inner BuildRight (8)\n :- BatchScan (1)\n +- BroadcastExchange (7)\n +- BatchScan (2)\n\n\n(1) BatchScan\nOutput [3]: [username#15, firstname#16, lastname#17]\nlocal.developers (branch=null) [filters=username IS NOT NULL, groupedBy=]\n\n(2) BatchScan\nOutput [2]: [creator#18, projectname#19]\nlocal.projects (branch=null) [filters=creator IS NOT NULL, groupedBy=]\n\n(3) ColumnarToRow [codegen id : 1]\nInput [2]: [creator#18, projectname#19]\n\n(4) BroadcastExchange\nInput [2]: [creator#18, projectname#19]\nArguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=75]\n\n(5) BroadcastQueryStage\nOutput [2]: [creator#18, projectname#19]\nArguments: 0\n\n(6) BroadcastHashJoin\nLeft keys [1]: [username#15]\nRight keys [1]: [creator#18]\nJoin condition: None\n\n(7) BroadcastExchange\nInput [2]: [creator#18, projectname#19]\nArguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=66]\n\n(8) BroadcastHashJoin\nLeft keys [1]: [username#15]\nRight keys [1]: [creator#18]\nJoin condition: None\n\n(9) AdaptiveSparkPlan\nOutput [5]: [username#15, firstname#16, lastname#17, creator#18, projectname#19]\nArguments: isFinalPlan=false\n\n","sparkPlanInfo":{"nodeName":"AdaptiveSparkPlan","simpleString":"AdaptiveSparkPlan isFinalPlan=false","children":[{"nodeName":"BroadcastHashJoin","simpleString":"BroadcastHashJoin [username#15], [creator#18], Inner, BuildRight, false","children":[{"nodeName":"BatchScan","simpleString":"BatchScan[username#15, firstname#16, lastname#17] local.developers (branch=null) [filters=username IS NOT NULL, groupedBy=] RuntimeFilters: []","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":131,"metricType":"sum"},{"name":"number of file splits read","accumulatorId":132,"metricType":"v2Custom_org.apache.iceberg.spark.source.metrics.NumSplits"},{"name":"number of row deletes applied","accumulatorId":133,"metricType":"v2Custom_org.apache.iceberg.spark.source.metrics.NumDeletes"}]},{"nodeName":"BroadcastQueryStage","simpleString":"BroadcastQueryStage 0","children":[{"nodeName":"BroadcastExchange","simpleString":"BroadcastExchange HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=75]","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"ColumnarToRow","simpleString":"ColumnarToRow","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"BatchScan","simpleString":"BatchScan[creator#18, projectname#19] local.projects (branch=null) [filters=creator IS NOT NULL, groupedBy=] RuntimeFilters: []","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":139,"metricType":"sum"},{"name":"number of file splits read","accumulatorId":140,"metricType":"v2Custom_org.apache.iceberg.spark.source.metrics.NumSplits"},{"name":"number of row deletes applied","accumulatorId":141,"metricType":"v2Custom_org.apache.iceberg.spark.source.metrics.NumDeletes"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":149,"metricType":"sum"},{"name":"number of input batches","accumulatorId":150,"metricType":"sum"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":148,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"time to broadcast","accumulatorId":147,"metricType":"timing"},{"name":"time to build","accumulatorId":146,"metricType":"timing"},{"name":"time to collect","accumulatorId":145,"metricType":"timing"},{"name":"number of output rows","accumulatorId":144,"metricType":"sum"},{"name":"data size","accumulatorId":143,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":142,"metricType":"sum"}]}],"metadata":{},"metrics":[]}} +{"Event":"SparkListenerJobStart","Job ID":4,"Submission Time":1714515315452,"Stage Infos":[{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:264","Number of Tasks":1,"RDD Info":[{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"BroadcastExchange\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:264","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"22\",\"name\":\"BatchScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:264","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"DataSourceRDD","Scope":"{\"id\":\"22\",\"name\":\"BatchScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:264","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:264","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:191)\njava.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)\njava.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)\njava.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)\njava.base/java.lang.Thread.run(Thread.java:840)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[6],"Properties":{"spark.dynamicAllocation.shuffleTracking.enabled":"true","spark.sql.warehouse.dir":"file:/home/holden/repos/sparklens/spark-warehouse","spark.executor.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.driver.host":"10.69.200.180","spark.eventLog.enabled":"true","spark.job.interruptOnCancel":"true","spark.driver.port":"37323","__fetch_continuous_blocks_in_batch_enabled":"true","spark.sql.catalog.local.type":"hadoop","spark.jars":"","spark.sql.catalog.spark_catalog":"org.apache.iceberg.spark.SparkSessionCatalog","spark.kubernetes.authenticate.submission.caCertFile":"/var/snap/microk8s/current/certs/ca.crt","spark.sql.datetime.java8API.enabled":"true","spark.sql.hive.version":"2.3.9","spark.app.name":"fresh-kube","spark.rdd.scope":"{\"id\":\"18\",\"name\":\"BroadcastExchange\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.submitTime":"1714515303940","spark.job.description":"broadcast exchange (runId 6b687057-1c02-48a5-ac06-2c12a5f49835)","spark.kubernetes.container.image":"localhost:32000/local-spark/spark:magic","spark.app.startTime":"1714515304235","spark.executor.id":"driver","spark.sql.catalog.local.warehouse":"/tmp/warehouse","spark.jobGroup.id":"6b687057-1c02-48a5-ac06-2c12a5f49835","spark.driver.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.sql.catalog.local":"org.apache.iceberg.spark.SparkCatalog","spark.submit.deployMode":"client","spark.master":"k8s://https://127.0.0.1:16443","spark.kubernetes.executor.podNamePrefix":"fresh-kube-bb0c5b8f311360be","spark.dynamicAllocation.enabled":"true","spark.sql.execution.id":"8","spark.sql.catalogImplementation":"hive","spark.kubernetes.authenticate.caCertFile":"/var/snap/microk8s/current/certs/ca.crt","spark.sql.catalog.spark_catalog.type":"hive","spark.app.id":"spark-a8d39974ec7b451a92b303fa953434e4"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:264","Number of Tasks":1,"RDD Info":[{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"BroadcastExchange\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:264","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"22\",\"name\":\"BatchScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:264","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"DataSourceRDD","Scope":"{\"id\":\"22\",\"name\":\"BatchScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:264","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:264","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:191)\njava.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)\njava.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)\njava.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)\njava.base/java.lang.Thread.run(Thread.java:840)","Submission Time":1714515315453,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.dynamicAllocation.shuffleTracking.enabled":"true","spark.sql.warehouse.dir":"file:/home/holden/repos/sparklens/spark-warehouse","spark.executor.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.driver.host":"10.69.200.180","spark.eventLog.enabled":"true","spark.job.interruptOnCancel":"true","spark.driver.port":"37323","__fetch_continuous_blocks_in_batch_enabled":"true","spark.sql.catalog.local.type":"hadoop","spark.jars":"","spark.sql.catalog.spark_catalog":"org.apache.iceberg.spark.SparkSessionCatalog","spark.kubernetes.authenticate.submission.caCertFile":"/var/snap/microk8s/current/certs/ca.crt","spark.sql.datetime.java8API.enabled":"true","spark.sql.hive.version":"2.3.9","spark.app.name":"fresh-kube","spark.rdd.scope":"{\"id\":\"18\",\"name\":\"BroadcastExchange\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.submitTime":"1714515303940","spark.job.description":"broadcast exchange (runId 6b687057-1c02-48a5-ac06-2c12a5f49835)","spark.kubernetes.container.image":"localhost:32000/local-spark/spark:magic","spark.app.startTime":"1714515304235","spark.executor.id":"driver","spark.sql.catalog.local.warehouse":"/tmp/warehouse","spark.jobGroup.id":"6b687057-1c02-48a5-ac06-2c12a5f49835","spark.driver.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.sql.catalog.local":"org.apache.iceberg.spark.SparkCatalog","spark.submit.deployMode":"client","spark.master":"k8s://https://127.0.0.1:16443","spark.kubernetes.executor.podNamePrefix":"fresh-kube-bb0c5b8f311360be","spark.dynamicAllocation.enabled":"true","spark.sql.execution.id":"8","spark.sql.catalogImplementation":"hive","spark.kubernetes.authenticate.caCertFile":"/var/snap/microk8s/current/certs/ca.crt","spark.sql.catalog.spark_catalog.type":"hive","spark.app.id":"spark-a8d39974ec7b451a92b303fa953434e4"}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":402,"Index":0,"Attempt":0,"Partition ID":0,"Launch Time":1714515315456,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":402,"Index":0,"Attempt":0,"Partition ID":0,"Launch Time":1714515315456,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315720,"Failed":false,"Killed":false,"Accumulables":[{"ID":139,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":140,"Name":"number of file splits read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":148,"Name":"duration","Update":"175","Value":"175","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":149,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":150,"Name":"number of input batches","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":151,"Name":"internal.metrics.executorDeserializeTime","Update":61,"Value":61,"Internal":true,"Count Failed Values":true},{"ID":152,"Name":"internal.metrics.executorDeserializeCpuTime","Update":58342386,"Value":58342386,"Internal":true,"Count Failed Values":true},{"ID":153,"Name":"internal.metrics.executorRunTime","Update":194,"Value":194,"Internal":true,"Count Failed Values":true},{"ID":154,"Name":"internal.metrics.executorCpuTime","Update":191312552,"Value":191312552,"Internal":true,"Count Failed Values":true},{"ID":155,"Name":"internal.metrics.resultSize","Update":1949,"Value":1949,"Internal":true,"Count Failed Values":true},{"ID":172,"Name":"internal.metrics.input.bytesRead","Update":1629,"Value":1629,"Internal":true,"Count Failed Values":true},{"ID":173,"Name":"internal.metrics.input.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":61,"Executor Deserialize CPU Time":58342386,"Executor Run Time":194,"Executor CPU Time":191312552,"Peak Execution Memory":0,"Result Size":1949,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":1629,"Records Read":1},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:264","Number of Tasks":1,"RDD Info":[{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"BroadcastExchange\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:264","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"22\",\"name\":\"BatchScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:264","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"DataSourceRDD","Scope":"{\"id\":\"22\",\"name\":\"BatchScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:264","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:264","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:191)\njava.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)\njava.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)\njava.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)\njava.base/java.lang.Thread.run(Thread.java:840)","Submission Time":1714515315453,"Completion Time":1714515315721,"Accumulables":[{"ID":139,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":140,"Name":"number of file splits read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":148,"Name":"duration","Value":"175","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":149,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":150,"Name":"number of input batches","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":151,"Name":"internal.metrics.executorDeserializeTime","Value":61,"Internal":true,"Count Failed Values":true},{"ID":152,"Name":"internal.metrics.executorDeserializeCpuTime","Value":58342386,"Internal":true,"Count Failed Values":true},{"ID":153,"Name":"internal.metrics.executorRunTime","Value":194,"Internal":true,"Count Failed Values":true},{"ID":154,"Name":"internal.metrics.executorCpuTime","Value":191312552,"Internal":true,"Count Failed Values":true},{"ID":155,"Name":"internal.metrics.resultSize","Value":1949,"Internal":true,"Count Failed Values":true},{"ID":172,"Name":"internal.metrics.input.bytesRead","Value":1629,"Internal":true,"Count Failed Values":true},{"ID":173,"Name":"internal.metrics.input.recordsRead","Value":1,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":4,"Completion Time":1714515315721,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":8,"accumUpdates":[[147,3],[146,15],[145,320],[144,1],[143,1049584]]} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate","executionId":8,"physicalPlanDescription":"== Physical Plan ==\nAdaptiveSparkPlan (10)\n+- == Final Plan ==\n * BroadcastHashJoin Inner BuildRight (7)\n :- * ColumnarToRow (2)\n : +- BatchScan (1)\n +- BroadcastQueryStage (6), Statistics(sizeInBytes=1025.0 KiB, rowCount=1)\n +- BroadcastExchange (5)\n +- * ColumnarToRow (4)\n +- BatchScan (3)\n+- == Initial Plan ==\n BroadcastHashJoin Inner BuildRight (9)\n :- BatchScan (1)\n +- BroadcastExchange (8)\n +- BatchScan (3)\n\n\n(1) BatchScan\nOutput [3]: [username#15, firstname#16, lastname#17]\nlocal.developers (branch=null) [filters=username IS NOT NULL, groupedBy=]\n\n(2) ColumnarToRow [codegen id : 2]\nInput [3]: [username#15, firstname#16, lastname#17]\n\n(3) BatchScan\nOutput [2]: [creator#18, projectname#19]\nlocal.projects (branch=null) [filters=creator IS NOT NULL, groupedBy=]\n\n(4) ColumnarToRow [codegen id : 1]\nInput [2]: [creator#18, projectname#19]\n\n(5) BroadcastExchange\nInput [2]: [creator#18, projectname#19]\nArguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=75]\n\n(6) BroadcastQueryStage\nOutput [2]: [creator#18, projectname#19]\nArguments: 0\n\n(7) BroadcastHashJoin [codegen id : 2]\nLeft keys [1]: [username#15]\nRight keys [1]: [creator#18]\nJoin condition: None\n\n(8) BroadcastExchange\nInput [2]: [creator#18, projectname#19]\nArguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=66]\n\n(9) BroadcastHashJoin\nLeft keys [1]: [username#15]\nRight keys [1]: [creator#18]\nJoin condition: None\n\n(10) AdaptiveSparkPlan\nOutput [5]: [username#15, firstname#16, lastname#17, creator#18, projectname#19]\nArguments: isFinalPlan=true\n\n","sparkPlanInfo":{"nodeName":"AdaptiveSparkPlan","simpleString":"AdaptiveSparkPlan isFinalPlan=true","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"BroadcastHashJoin","simpleString":"BroadcastHashJoin [username#15], [creator#18], Inner, BuildRight, false","children":[{"nodeName":"ColumnarToRow","simpleString":"ColumnarToRow","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"BatchScan","simpleString":"BatchScan[username#15, firstname#16, lastname#17] local.developers (branch=null) [filters=username IS NOT NULL, groupedBy=] RuntimeFilters: []","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":131,"metricType":"sum"},{"name":"number of file splits read","accumulatorId":132,"metricType":"v2Custom_org.apache.iceberg.spark.source.metrics.NumSplits"},{"name":"number of row deletes applied","accumulatorId":133,"metricType":"v2Custom_org.apache.iceberg.spark.source.metrics.NumDeletes"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":178,"metricType":"sum"},{"name":"number of input batches","accumulatorId":179,"metricType":"sum"}]},{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"BroadcastQueryStage","simpleString":"BroadcastQueryStage 0","children":[{"nodeName":"BroadcastExchange","simpleString":"BroadcastExchange HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=75]","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"ColumnarToRow","simpleString":"ColumnarToRow","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"BatchScan","simpleString":"BatchScan[creator#18, projectname#19] local.projects (branch=null) [filters=creator IS NOT NULL, groupedBy=] RuntimeFilters: []","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":139,"metricType":"sum"},{"name":"number of file splits read","accumulatorId":140,"metricType":"v2Custom_org.apache.iceberg.spark.source.metrics.NumSplits"},{"name":"number of row deletes applied","accumulatorId":141,"metricType":"v2Custom_org.apache.iceberg.spark.source.metrics.NumDeletes"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":149,"metricType":"sum"},{"name":"number of input batches","accumulatorId":150,"metricType":"sum"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":148,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"time to broadcast","accumulatorId":147,"metricType":"timing"},{"name":"time to build","accumulatorId":146,"metricType":"timing"},{"name":"time to collect","accumulatorId":145,"metricType":"timing"},{"name":"number of output rows","accumulatorId":144,"metricType":"sum"},{"name":"data size","accumulatorId":143,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":177,"metricType":"sum"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":176,"metricType":"timing"}]}],"metadata":{},"metrics":[]}} +{"Event":"SparkListenerJobStart","Job ID":5,"Submission Time":1714515315781,"Stage Infos":[{"Stage ID":7,"Stage Attempt ID":0,"Stage Name":"processLine at CliDriver.java:336","Number of Tasks":1,"RDD Info":[{"RDD ID":15,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"36\",\"name\":\"mapPartitionsInternal\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[14],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"DataSourceRDD","Scope":"{\"id\":\"35\",\"name\":\"BatchScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"35\",\"name\":\"BatchScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"27\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[7],"Properties":{"spark.dynamicAllocation.shuffleTracking.enabled":"true","spark.sql.warehouse.dir":"file:/home/holden/repos/sparklens/spark-warehouse","spark.executor.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.driver.host":"10.69.200.180","spark.eventLog.enabled":"true","spark.driver.port":"37323","__fetch_continuous_blocks_in_batch_enabled":"true","spark.sql.catalog.local.type":"hadoop","spark.jars":"","spark.sql.catalog.spark_catalog":"org.apache.iceberg.spark.SparkSessionCatalog","spark.kubernetes.authenticate.submission.caCertFile":"/var/snap/microk8s/current/certs/ca.crt","spark.sql.datetime.java8API.enabled":"true","spark.sql.hive.version":"2.3.9","spark.app.name":"fresh-kube","spark.rdd.scope":"{\"id\":\"37\",\"name\":\"collect\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.submitTime":"1714515303940","spark.job.description":"\nSELECT * FROM local.developers INNER JOIN local.projects ON local.projects.creator = local.developers.username","spark.kubernetes.container.image":"localhost:32000/local-spark/spark:magic","spark.app.startTime":"1714515304235","spark.executor.id":"driver","spark.sql.catalog.local.warehouse":"/tmp/warehouse","spark.driver.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.sql.catalog.local":"org.apache.iceberg.spark.SparkCatalog","spark.submit.deployMode":"client","spark.master":"k8s://https://127.0.0.1:16443","spark.kubernetes.executor.podNamePrefix":"fresh-kube-bb0c5b8f311360be","spark.dynamicAllocation.enabled":"true","spark.sql.execution.id":"8","spark.sql.catalogImplementation":"hive","spark.kubernetes.authenticate.caCertFile":"/var/snap/microk8s/current/certs/ca.crt","spark.sql.catalog.spark_catalog.type":"hive","spark.app.id":"spark-a8d39974ec7b451a92b303fa953434e4"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":7,"Stage Attempt ID":0,"Stage Name":"processLine at CliDriver.java:336","Number of Tasks":1,"RDD Info":[{"RDD ID":15,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"36\",\"name\":\"mapPartitionsInternal\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[14],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"DataSourceRDD","Scope":"{\"id\":\"35\",\"name\":\"BatchScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"35\",\"name\":\"BatchScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"27\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1714515315781,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.dynamicAllocation.shuffleTracking.enabled":"true","spark.sql.warehouse.dir":"file:/home/holden/repos/sparklens/spark-warehouse","spark.executor.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.driver.host":"10.69.200.180","spark.eventLog.enabled":"true","spark.driver.port":"37323","__fetch_continuous_blocks_in_batch_enabled":"true","spark.sql.catalog.local.type":"hadoop","spark.jars":"","spark.sql.catalog.spark_catalog":"org.apache.iceberg.spark.SparkSessionCatalog","spark.kubernetes.authenticate.submission.caCertFile":"/var/snap/microk8s/current/certs/ca.crt","spark.sql.datetime.java8API.enabled":"true","spark.sql.hive.version":"2.3.9","spark.app.name":"fresh-kube","spark.rdd.scope":"{\"id\":\"37\",\"name\":\"collect\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.submitTime":"1714515303940","spark.job.description":"\nSELECT * FROM local.developers INNER JOIN local.projects ON local.projects.creator = local.developers.username","spark.kubernetes.container.image":"localhost:32000/local-spark/spark:magic","spark.app.startTime":"1714515304235","spark.executor.id":"driver","spark.sql.catalog.local.warehouse":"/tmp/warehouse","spark.driver.extraJavaOptions":"-XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED","spark.sql.catalog.local":"org.apache.iceberg.spark.SparkCatalog","spark.submit.deployMode":"client","spark.master":"k8s://https://127.0.0.1:16443","spark.kubernetes.executor.podNamePrefix":"fresh-kube-bb0c5b8f311360be","spark.dynamicAllocation.enabled":"true","spark.sql.execution.id":"8","spark.sql.catalogImplementation":"hive","spark.kubernetes.authenticate.caCertFile":"/var/snap/microk8s/current/certs/ca.crt","spark.sql.catalog.spark_catalog.type":"hive","spark.app.id":"spark-a8d39974ec7b451a92b303fa953434e4"}} +{"Event":"SparkListenerTaskStart","Stage ID":7,"Stage Attempt ID":0,"Task Info":{"Task ID":403,"Index":0,"Attempt":0,"Partition ID":0,"Launch Time":1714515315785,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":7,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":403,"Index":0,"Attempt":0,"Partition ID":0,"Launch Time":1714515315785,"Executor ID":"1","Host":"10.1.246.124","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1714515315828,"Failed":false,"Killed":false,"Accumulables":[{"ID":131,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":132,"Name":"number of file splits read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":176,"Name":"duration","Update":"18","Value":"18","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":177,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":178,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":179,"Name":"number of input batches","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":180,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":5,"Internal":true,"Count Failed Values":true},{"ID":181,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3454491,"Value":3454491,"Internal":true,"Count Failed Values":true},{"ID":182,"Name":"internal.metrics.executorRunTime","Update":36,"Value":36,"Internal":true,"Count Failed Values":true},{"ID":183,"Name":"internal.metrics.executorCpuTime","Update":31882421,"Value":31882421,"Internal":true,"Count Failed Values":true},{"ID":184,"Name":"internal.metrics.resultSize","Update":2081,"Value":2081,"Internal":true,"Count Failed Values":true},{"ID":189,"Name":"internal.metrics.peakExecutionMemory","Update":16778240,"Value":16778240,"Internal":true,"Count Failed Values":true},{"ID":201,"Name":"internal.metrics.input.bytesRead","Update":2316,"Value":2316,"Internal":true,"Count Failed Values":true},{"ID":202,"Name":"internal.metrics.input.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0,"TotalGCTime":0},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":3454491,"Executor Run Time":36,"Executor CPU Time":31882421,"Peak Execution Memory":16778240,"Result Size":2081,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":2316,"Records Read":1},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":7,"Stage Attempt ID":0,"Stage Name":"processLine at CliDriver.java:336","Number of Tasks":1,"RDD Info":[{"RDD ID":15,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"36\",\"name\":\"mapPartitionsInternal\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[14],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"DataSourceRDD","Scope":"{\"id\":\"35\",\"name\":\"BatchScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"35\",\"name\":\"BatchScan\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"27\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"processLine at CliDriver.java:336","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"DeterministicLevel":"DETERMINATE","Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:498)\norg.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:336)\norg.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:474)\norg.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:490)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:212)\norg.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\njava.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:568)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:984)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:191)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:214)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1072)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1081)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1714515315781,"Completion Time":1714515315828,"Accumulables":[{"ID":131,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":132,"Name":"number of file splits read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":176,"Name":"duration","Value":"18","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":177,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":178,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":179,"Name":"number of input batches","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":180,"Name":"internal.metrics.executorDeserializeTime","Value":5,"Internal":true,"Count Failed Values":true},{"ID":181,"Name":"internal.metrics.executorDeserializeCpuTime","Value":3454491,"Internal":true,"Count Failed Values":true},{"ID":182,"Name":"internal.metrics.executorRunTime","Value":36,"Internal":true,"Count Failed Values":true},{"ID":183,"Name":"internal.metrics.executorCpuTime","Value":31882421,"Internal":true,"Count Failed Values":true},{"ID":184,"Name":"internal.metrics.resultSize","Value":2081,"Internal":true,"Count Failed Values":true},{"ID":189,"Name":"internal.metrics.peakExecutionMemory","Value":16778240,"Internal":true,"Count Failed Values":true},{"ID":201,"Name":"internal.metrics.input.bytesRead","Value":2316,"Internal":true,"Count Failed Values":true},{"ID":202,"Name":"internal.metrics.input.recordsRead","Value":1,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":5,"Completion Time":1714515315829,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":8,"time":1714515315831} +{"Event":"SparkListenerApplicationEnd","Timestamp":1714515315834} diff --git a/src/test/scala/CompatibilitySuite.scala b/src/test/scala/CompatibilitySuite.scala deleted file mode 100644 index 8439baa..0000000 --- a/src/test/scala/CompatibilitySuite.scala +++ /dev/null @@ -1,41 +0,0 @@ -import java.io.{ByteArrayOutputStream, FileNotFoundException, PrintStream} -import com.qubole.sparklens.{TestUtils, TimezoneSuite} -import com.qubole.sparklens.app.ReporterApp -import org.scalatest.FunSuite - -import scala.util.control.Breaks._ - -class CompatibilitySuite extends FunSuite with TimezoneSuite { - - test("should be able to report on previously generated sparklens dumps") { - - breakable { - - (1 to 100).foreach(x => { //run for the versions of sparklens output saved - try { - - val testInput = TestUtils.getFileContents( - s"${System.getProperty("user.dir")}/src/test/compatibility-files/version-${x}.json") - - val testOut = new ByteArrayOutputStream() - Console.withOut(new PrintStream(testOut)) { - ReporterApp.startAnalysersFromString(testInput) - } - val testOutput = testOut.toString - - val olderOutput = TestUtils.getFileContents( - s"${System.getProperty("user.dir")}/src/test/compatibility-files/version-${x}.output") - - /* checking that some important lines of the actual run also appear on - * running in this test using the sparklens dumps */ - olderOutput.split("\n").foreach(line => { - assert(testOutput.contains(line)) - }) - } catch { - case e: FileNotFoundException => break - } - }) - } - } - -} \ No newline at end of file diff --git a/src/test/scala/com/qubole/sparklens/TestUtils.scala b/src/test/scala/com/qubole/sparklens/TestUtils.scala index 8731d72..7557433 100644 --- a/src/test/scala/com/qubole/sparklens/TestUtils.scala +++ b/src/test/scala/com/qubole/sparklens/TestUtils.scala @@ -1,6 +1,7 @@ package com.qubole.sparklens -import org.scalatest.{BeforeAndAfterAll, FunSuite} +import org.scalatest.BeforeAndAfterAll +import org.scalatest.funsuite.AnyFunSuite import java.util.TimeZone import scala.io.Source @@ -15,7 +16,7 @@ object TestUtils { } -trait TimezoneSuite extends FunSuite with BeforeAndAfterAll { +trait TimezoneSuite extends AnyFunSuite with BeforeAndAfterAll { val defaultTimeZone: TimeZone = TimeZone.getDefault override def beforeAll(): Unit = { @@ -25,4 +26,4 @@ trait TimezoneSuite extends FunSuite with BeforeAndAfterAll { override def afterAll(): Unit = { TimeZone.setDefault(defaultTimeZone) } -} \ No newline at end of file +} diff --git a/src/test/scala/com/qubole/sparklens/analyzer/ExecutorTimelineAnalyzerSuite.scala b/src/test/scala/com/qubole/sparklens/analyzer/ExecutorTimelineAnalyzerSuite.scala new file mode 100644 index 0000000..aac8b4f --- /dev/null +++ b/src/test/scala/com/qubole/sparklens/analyzer/ExecutorTimelineAnalyzerSuite.scala @@ -0,0 +1,77 @@ +/* +* 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. +*/ + +package com.qubole.sparklens.analyzer + +import com.qubole.sparklens.common.{AggregateMetrics, AppContext, ApplicationInfo} +import com.qubole.sparklens.timespan.{ExecutorTimeSpan, HostTimeSpan, JobTimeSpan, StageTimeSpan} +import com.qubole.sparklens.helper.JobOverlapHelper + +import org.scalatest.funsuite.AnyFunSuite + +import org.apache.spark.SparkConf + +import scala.collection.mutable + +class ExecutorTimelineAnalyzerSuite extends AnyFunSuite { + + val startTime = 0 + val endTime = 60000000000L + def createDummyAppContext(): AppContext = { + + val jobMap = new mutable.HashMap[Long, JobTimeSpan] + + val jobSQLExecIDMap = new mutable.HashMap[Long, Long] + + val execStartTimes = new mutable.HashMap[String, ExecutorTimeSpan]() + + val appInfo = new ApplicationInfo() + appInfo.startTime = startTime + appInfo.endTime = endTime + + val firstExec = new ExecutorTimeSpan("1", "1", 1) + firstExec.setStartTime(0) + firstExec.setEndTime(6000000) + execStartTimes("1") = firstExec + // We start an 2nd exec but exit "right away". + val secondExec = new ExecutorTimeSpan("2", "1", 1) + secondExec.setStartTime(0) + secondExec.setEndTime(120) + execStartTimes("2") = secondExec + + + val conf = new SparkConf().set("spark.dynamicAllocation.enabled", "true") + + new AppContext(appInfo, + new AggregateMetrics(), + mutable.HashMap[String, HostTimeSpan](), + execStartTimes, + jobMap, + jobSQLExecIDMap, + mutable.HashMap[Int, StageTimeSpan](), + mutable.HashMap[Int, Long](), + Some(conf.getAll.toMap)) + } + + test("A reasonable guess at initial executors") { + val ac = createDummyAppContext() + val eta = new ExecutorTimelineAnalyzer() + val (logs, suggestions) = eta.analyzeAndSuggest(ac, startTime, endTime) + assert(suggestions.get("spark.dynamicAllocation.initialExecutors") == Some("1"), + "Initial exec suggestions") + } +} diff --git a/src/test/scala/com/qubole/sparklens/analyzer/JobOverlapAnalyzerSuite.scala b/src/test/scala/com/qubole/sparklens/analyzer/JobOverlapAnalyzerSuite.scala index f129413..e2f2642 100644 --- a/src/test/scala/com/qubole/sparklens/analyzer/JobOverlapAnalyzerSuite.scala +++ b/src/test/scala/com/qubole/sparklens/analyzer/JobOverlapAnalyzerSuite.scala @@ -1,4 +1,3 @@ - /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -22,11 +21,13 @@ import com.qubole.sparklens.common.{AggregateMetrics, AppContext, ApplicationInf import com.qubole.sparklens.timespan.{ExecutorTimeSpan, HostTimeSpan, JobTimeSpan, StageTimeSpan} import com.qubole.sparklens.helper.JobOverlapHelper -import org.scalatest.FunSuite +import org.scalatest.funsuite.AnyFunSuite + +import org.apache.spark.SparkConf import scala.collection.mutable -class JobOverlapAnalyzerSuite extends FunSuite { +class JobOverlapAnalyzerSuite extends AnyFunSuite { def createDummyAppContext(): AppContext = { @@ -66,7 +67,8 @@ class JobOverlapAnalyzerSuite extends FunSuite { jobMap, jobSQLExecIDMap, mutable.HashMap[Int, StageTimeSpan](), - mutable.HashMap[Int, Long]()) + mutable.HashMap[Int, Long](), + Some(new SparkConf().getAll.toMap)) } test("JobOverlapAnalyzerTest: Jobs running in parallel should be considered while computing " + diff --git a/src/test/scala/com/qubole/sparklens/app/EventHistoryTest.scala b/src/test/scala/com/qubole/sparklens/app/EventHistoryTest.scala new file mode 100644 index 0000000..61c86c6 --- /dev/null +++ b/src/test/scala/com/qubole/sparklens/app/EventHistoryTest.scala @@ -0,0 +1,60 @@ +package com.qubole.sparklens.app + +import java.io.{ByteArrayOutputStream, PrintStream} + +import com.qubole.sparklens.TestUtils +import org.scalatest.funsuite.AnyFunSuite +import org.scalatest.matchers.should._ + +class EventHistoryFileReportingSuite extends AnyFunSuite with Matchers { + + test("Reporting from our static sparklens should be reasonable") { + val eventHistoryFile = s"${System.getProperty("user.dir")}" + + s"/src/test/event-history-test-files/local-1532512550423" + + validateOutput(outputFromEventHistoryReport(eventHistoryFile), + appId = "local-1532512550423", + updatedConf = Map.empty[String, String]) + } + + test("Reporting from our fresh sparklens should be reasonable") { + val eventHistoryFile = s"${System.getProperty("user.dir")}" + + s"/src/test/event-history-test-files/local-fresh" + + validateOutput(outputFromEventHistoryReport(eventHistoryFile), + appId = "local-", + updatedConf = Map.empty[String, String]) + } + + test("Reporting from our dynamically scaled & fresh sparklens should be reasonable") { + val eventHistoryFile = s"${System.getProperty("user.dir")}" + + s"/src/test/event-history-test-files/local-fresh-dynamic" + + validateOutput(outputFromEventHistoryReport(eventHistoryFile), + appId = "spark-", + updatedConf = Map.empty[String, String], + dynamic=true) + } + + private def outputFromEventHistoryReport(file: String): String = { + val out = new ByteArrayOutputStream() + Console.withOut(new PrintStream(out)) { + new EventHistoryReporter(file) + } + out.toString + } + + private def validateOutput( + fileContents: String, + appId: String, + updatedConf: Map[String, String], + dynamic: Boolean = false, + constantExecs: Boolean = true) = { + fileContents should include(s"ID $appId") + if (dynamic && constantExecs) { + fileContents should include("Initial execs within tolerance +-10% leaving alone") + } else { + fileContents should include("since not dynamic") + } + } +} diff --git a/src/test/scala/com/qubole/sparklens/common/AppContextSuite.scala b/src/test/scala/com/qubole/sparklens/common/AppContextSuite.scala index 1afc771..917ce2c 100644 --- a/src/test/scala/com/qubole/sparklens/common/AppContextSuite.scala +++ b/src/test/scala/com/qubole/sparklens/common/AppContextSuite.scala @@ -1,9 +1,10 @@ package com.qubole.sparklens.common import com.qubole.sparklens.timespan.{ExecutorTimeSpan, StageTimeSpan} -import org.scalatest.FunSuite +import org.scalatest.funsuite.AnyFunSuite -class AppContextSuite extends FunSuite { + +class AppContextSuite extends AnyFunSuite { test ("maximum concurrent executors at a time") { diff --git a/src/test/scala/com/qubole/sparklens/scheduler/PQParallelStageSchedulerSuite.scala b/src/test/scala/com/qubole/sparklens/scheduler/PQParallelStageSchedulerSuite.scala index 86fbe77..fd109ef 100644 --- a/src/test/scala/com/qubole/sparklens/scheduler/PQParallelStageSchedulerSuite.scala +++ b/src/test/scala/com/qubole/sparklens/scheduler/PQParallelStageSchedulerSuite.scala @@ -19,12 +19,12 @@ package com.qubole.sparklens.scheduler import com.qubole.sparklens.common.{AggregateMetrics, AppContext, ApplicationInfo} import com.qubole.sparklens.timespan.{ExecutorTimeSpan, HostTimeSpan, JobTimeSpan, StageTimeSpan} -import org.scalatest.FunSuite +import org.scalatest.funsuite.AnyFunSuite import scala.collection.mutable import scala.collection.mutable.ListBuffer -class PQParallelStageSchedulerSuite extends FunSuite { +class PQParallelStageSchedulerSuite extends AnyFunSuite { def createStageTimeSpan(stageID: Int, taskCount: Int, minTaskLaunchTime: Long, maxTaskFinishTime: Long, parentStages: Seq[Int]): StageTimeSpan = { @@ -117,7 +117,7 @@ class PQParallelStageSchedulerSuite extends FunSuite { parentList.append(i) } //make the final stage - listBuffer.append((n, 1, n.toLong, (n + 1).toLong, parentList)) + listBuffer.append((n, 1, n.toLong, (n + 1).toLong, parentList.toList)) listBuffer.toList } @@ -266,7 +266,8 @@ class PQParallelStageSchedulerSuite extends FunSuite { jobMap, jobSQLExecIDMap, mutable.HashMap[Int, StageTimeSpan](), - mutable.HashMap[Int, Long]()) + mutable.HashMap[Int, Long](), + Some(Map.empty[String, String])) val time = CompletionEstimator.estimateAppWallClockTimeWithJobLists(ac, 1, 1, 3) assert(time === 3, s"Test failed")