diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala index 42647f880..a3e6139b0 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala @@ -175,7 +175,8 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs, enablePB: Boolea processSuccessApp(app) progressBar.foreach(_.reportSuccessfulProcess()) val endTime = System.currentTimeMillis() - SuccessAppResult(pathStr, app.appId, s"Took ${endTime - startTime}ms to process") + SuccessAppResult(pathStr, app.appId, app.attemptId, + s"Took ${endTime - startTime}ms to process") } // Log the information to the console profAppResult.logMessage() diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/AppSubscriber.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/AppSubscriber.scala new file mode 100644 index 000000000..5bcbfdeb5 --- /dev/null +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/AppSubscriber.scala @@ -0,0 +1,72 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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.nvidia.spark.rapids.tool.qualification + +class AppSubscriber(val appId: String) { + val lock = new Object() + private var attemptID: Option[Int] = None + + def unsafeSetAttemptId(newAttempt: Int): Boolean = { + attemptID match { + case Some(a) => + if (newAttempt > a) { + attemptID = Some(newAttempt) + } + case None => attemptID = Some(newAttempt) + } + newAttempt == attemptID.get + } + + def safeSetAttemptId(newAttempt: Int): Boolean = { + lock.synchronized { + unsafeSetAttemptId(newAttempt) + } + } +} + +object AppSubscriber { + private val APP_SUBSCRIBERS = new java.util.concurrent.ConcurrentHashMap[String, AppSubscriber]() + + def getOrCreate(appId: String): AppSubscriber = { + APP_SUBSCRIBERS.computeIfAbsent(appId, _ => new AppSubscriber(appId)) + } + + def subscribeAppAttempt(appId: String, newAttemptId: Int): Boolean = { + val subscriber = getOrCreate(appId) + subscriber.safeSetAttemptId(newAttemptId) + } + + def withSafeValidAttempt[T](appId: String, currAttempt: Int)(f: () => T): Option[T] = { + val subscriber = getOrCreate(appId) + subscriber.lock.synchronized { + if (subscriber.unsafeSetAttemptId(currAttempt)) { + Option(f()) + } else { + None + } + } + } + + def withUnsafeValidAttempt[T](appId: String, currAttempt: Int)(f: () => T): Option[T] = { + val subscriber = getOrCreate(appId) + if (subscriber.unsafeSetAttemptId(currAttempt)) { + Option(f()) + } else { + None + } + } +} diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/Qualification.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/Qualification.scala index f21568bed..6889116ea 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/Qualification.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/Qualification.scala @@ -16,7 +16,7 @@ package com.nvidia.spark.rapids.tool.qualification -import java.util.concurrent.{ConcurrentLinkedQueue, TimeUnit} +import java.util.concurrent.{ConcurrentHashMap, TimeUnit} import scala.collection.JavaConverters._ @@ -41,7 +41,7 @@ class Qualification(outputPath: String, numRows: Int, hadoopConf: Configuration, override val simpleName: String = "qualTool" override val outputDir = s"$outputPath/rapids_4_spark_qualification_output" - private val allApps = new ConcurrentLinkedQueue[QualificationSummaryInfo]() + private val allApps = new ConcurrentHashMap[String, QualificationSummaryInfo]() override def getNumThreads: Int = nThreads @@ -72,7 +72,7 @@ class Qualification(outputPath: String, numRows: Int, hadoopConf: Configuration, threadPool.shutdownNow() } progressBar.foreach(_.finishAll()) - val allAppsSum = estimateAppFrequency(allApps.asScala.toSeq) + val allAppsSum = estimateAppFrequency(allApps.asScala.values.toSeq) // sort order and limit only applies to the report summary text file, // the csv file we write the entire data in descending order val sortedDescDetailed = sortDescForDetailedReport(allAppsSum) @@ -162,27 +162,48 @@ class Qualification(outputPath: String, numRows: Int, hadoopConf: Configuration, // this is a bit ugly right now to overload writing out the report and returning the // DataSource information but this encapsulates the analyzer to keep the memory usage // smaller. - val dsInfo = QualRawReportGenerator.generateRawMetricQualViewAndGetDataSourceInfo( - outputDir, app, appIndex) + val dsInfo = + AppSubscriber.withSafeValidAttempt(app.appId, app.attemptId) { () => + QualRawReportGenerator.generateRawMetricQualViewAndGetDataSourceInfo( + outputDir, app, appIndex) + }.getOrElse(Seq.empty) val qualSumInfo = app.aggregateStats() - tunerContext.foreach { tuner => - // Run the autotuner if it is enabled. - // Note that we call the autotuner anyway without checking the aggregate results - // because the Autotuner can still make some recommendations based on the information - // enclosed by the QualificationInfo object - tuner.tuneApplication(app, qualSumInfo, appIndex, dsInfo) + AppSubscriber.withSafeValidAttempt(app.appId, app.attemptId) { () => + tunerContext.foreach { tuner => + // Run the autotuner if it is enabled. + // Note that we call the autotuner anyway without checking the aggregate results + // because the Autotuner can still make some recommendations based on the information + // enclosed by the QualificationInfo object + tuner.tuneApplication(app, qualSumInfo, appIndex, dsInfo) + } } if (qualSumInfo.isDefined) { // add the recommend cluster info into the summary val tempSummary = qualSumInfo.get val newClusterSummary = tempSummary.clusterSummary.copy( recommendedClusterInfo = pluginTypeChecker.platform.recommendedClusterInfo) - val newQualSummary = tempSummary.copy(clusterSummary = newClusterSummary) - allApps.add(newQualSummary) - progressBar.foreach(_.reportSuccessfulProcess()) - val endTime = System.currentTimeMillis() - SuccessAppResult(pathStr, app.appId, - s"Took ${endTime - startTime}ms to process") + AppSubscriber.withSafeValidAttempt(app.appId, app.attemptId) { () => + val newQualSummary = tempSummary.copy(clusterSummary = newClusterSummary) + // check if the app is already in the map + if (allApps.containsKey(app.appId)) { + // fix the progress bar counts + progressBar.foreach(_.adjustCounterForMultipleAttempts()) + logInfo(s"Removing older app summary for app: ${app.appId} " + + s"before adding the new one with attempt: ${app.attemptId}") + } + progressBar.foreach(_.reportSuccessfulProcess()) + allApps.put(app.appId, newQualSummary) + val endTime = System.currentTimeMillis() + SuccessAppResult(pathStr, app.appId, app.attemptId, + s"Took ${endTime - startTime}ms to process") + } match { + case Some(successfulResult) => successfulResult + case _ => + // If the attemptId is an older attemptId, skip this attempt. + // This can happen when the user has provided event logs for multiple attempts + progressBar.foreach(_.reportSkippedProcess()) + SkippedAppResult.fromAppAttempt(pathStr, app.appId, app.attemptId) + } } else { progressBar.foreach(_.reportUnkownStatusProcess()) UnknownAppResult(pathStr, app.appId, diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/AppBase.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/AppBase.scala index e62d1ba0d..a4db6b1a3 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/AppBase.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/AppBase.scala @@ -27,6 +27,7 @@ import com.nvidia.spark.rapids.tool.{DatabricksEventLog, DatabricksRollingEventL import com.nvidia.spark.rapids.tool.planparser.{HiveParseHelper, ReadParser} import com.nvidia.spark.rapids.tool.planparser.HiveParseHelper.isHiveTableScanNode import com.nvidia.spark.rapids.tool.profiling.{BlockManagerRemovedCase, DataSourceCase, DriverAccumCase, JobInfoClass, ResourceProfileInfoCase, SQLExecutionInfoClass, SQLPlanMetricsCase} +import com.nvidia.spark.rapids.tool.qualification.AppSubscriber import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} @@ -53,6 +54,8 @@ abstract class AppBase( } } + lazy val attemptId: Int = appMetaData.map(_.attemptId).getOrElse(1) + // Store map of executorId to executor info val executorIdToInfo = new HashMap[String, ExecutorInfoClass]() // resourceProfile id to resource profile info @@ -396,7 +399,20 @@ abstract class AppBase( probNotDataset.values.flatten.toSet.toSeq } + /** + * Registers the attempt ID for the application and updates the tracker map if the attemptId is + * greater than the existing attemptId. + */ + def registerAttemptId(): Unit = { + if (isAppMetaDefined) { + val currentAttemptId = sparkProperties.getOrElse("spark.app.attempt.id", "1").toInt + appMetaData.foreach(_.setAttemptId(currentAttemptId)) + AppSubscriber.subscribeAppAttempt(appId, currentAttemptId) + } + } + protected def postCompletion(): Unit = { + registerAttemptId() calculateAppDuration() } diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/AppMetaData.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/AppMetaData.scala index ef2c8e89c..41087d262 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/AppMetaData.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/AppMetaData.scala @@ -29,6 +29,7 @@ import org.apache.spark.ui.UIUtils * @param sparkUser user who ran the Spark application * @param startTime startTime of a Spark application * @param endTime endTime of the spark Application + * @param attemptId attemptId of the application */ class AppMetaData( val eventLogPath: Option[String], @@ -36,7 +37,8 @@ class AppMetaData( val appId: Option[String], val sparkUser: String, val startTime: Long, - var endTime: Option[Long] = None) { + var endTime: Option[Long] = None, + var attemptId: Int = 1) { // Calculated as time in ms var duration: Option[Long] = _ @@ -75,6 +77,10 @@ class AppMetaData( def isDurationEstimated: Boolean = durationEstimated + def setAttemptId(attemptId: Int): Unit = { + this.attemptId = attemptId + } + // Initialization code: // - Calculate the duration based on the constructor argument endTime calculateDurationInternal() diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/EventProcessorBase.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/EventProcessorBase.scala index e6593035e..e84a301a3 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/EventProcessorBase.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/EventProcessorBase.scala @@ -290,6 +290,7 @@ abstract class EventProcessorBase[T <: AppBase](app: T) extends SparkListener wi event: SparkListenerEnvironmentUpdate): Unit = { logDebug("Processing event: " + event.getClass) app.handleEnvUpdateForCachedProps(event) + app.registerAttemptId() } override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate): Unit = { diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/ui/ConsoleProgressBar.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/ui/ConsoleProgressBar.scala index 630368e94..67d3a83e1 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/ui/ConsoleProgressBar.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/ui/ConsoleProgressBar.scala @@ -119,6 +119,11 @@ class ConsoleProgressBar( } } + def adjustCounterForMultipleAttempts(): Unit = { + successCounter.decrementAndGet() + skippedCounter.incrementAndGet() + } + def reportSuccessfulProcess(): Unit = { successCounter.incrementAndGet() totalCounter.incrementAndGet() diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/util/OperationResult.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/util/OperationResult.scala index 0c60b574a..129872a0a 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/util/OperationResult.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/util/OperationResult.scala @@ -38,6 +38,7 @@ class AppResult(path: String, message: String) extends Logging { case class SuccessAppResult( path: String, appId: String, + attemptId: Int, message: String = "") extends AppResult(path, message) { override def logMessage(exp: Option[Exception] = None): Unit = { logInfo(s"File: $path, Message: $message") @@ -52,3 +53,10 @@ case class UnknownAppResult(path: String, appId: String, message: String) case class SkippedAppResult(path: String, message: String) extends AppResult(path, message) {} + +object SkippedAppResult { + def fromAppAttempt(path: String, appId: String, attemptId: Int): SkippedAppResult = { + SkippedAppResult(path, s"For App ID '$appId'; skipping this " + + s"attempt $attemptId as a newer attemptId is being processed") + } +} diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/util/RuntimeReporter.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/util/RuntimeReporter.scala index e0fb686fe..57a8ad074 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/util/RuntimeReporter.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/util/RuntimeReporter.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.rapids.tool.util import com.nvidia.spark.rapids.tool.profiling.AppStatusResult +import com.nvidia.spark.rapids.tool.qualification.AppSubscriber import org.apache.hadoop.conf.Configuration import org.apache.spark.internal.Logging @@ -27,18 +28,33 @@ trait RuntimeReporter extends Logging { RuntimeUtil.generateReport(outputDir, hadoopConf) } + /** + * Updates the status of "SUCCESS" applications to "SKIPPED" if newer attempts with + * the same appId exist. + */ + private def skipAppsWithOlderAttempts(appStatuses: Seq[AppResult]): Seq[AppResult] = { + appStatuses map { + case successApp: SuccessAppResult => + AppSubscriber.withUnsafeValidAttempt(successApp.appId, successApp.attemptId) { () => + successApp + }.getOrElse(SkippedAppResult.fromAppAttempt(successApp.path, successApp.appId, + successApp.attemptId)) + case otherApp: AppResult => otherApp + } + } + /** * For each app status report, generate an AppStatusResult. * If appId is empty, convert to "N/A" in the output. * @return Seq[AppStatusResult] - Seq[(path, status, appId, message)] */ def generateStatusResults(appStatuses: Seq[AppResult]): Seq[AppStatusResult] = { - appStatuses.map { + skipAppsWithOlderAttempts(appStatuses).map { case FailureAppResult(path, message) => AppStatusResult(path, "FAILURE", "N/A", message) case SkippedAppResult(path, message) => AppStatusResult(path, "SKIPPED", "N/A", message) - case SuccessAppResult(path, appId, message) => + case SuccessAppResult(path, appId, _, message) => AppStatusResult(path, "SUCCESS", appId, message) case UnknownAppResult(path, appId, message) => val finalAppId = if (appId.isEmpty) "N/A" else appId diff --git a/core/src/test/resources/spark-events-qualification/cluster_information/eventlog_2nodes_8cores b/core/src/test/resources/spark-events-qualification/cluster_information/eventlog_2nodes_8cores index dbfc21fe2..141d6a98e 100644 --- a/core/src/test/resources/spark-events-qualification/cluster_information/eventlog_2nodes_8cores +++ b/core/src/test/resources/spark-events-qualification/cluster_information/eventlog_2nodes_8cores @@ -5,11 +5,11 @@ {"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"10.59.184.228","Port":46767},"Maximum Memory":11884245811,"Timestamp":1702417716594,"Maximum Onheap Memory":11884245811,"Maximum Offheap Memory":0} {"Event":"SparkListenerExecutorAdded","Timestamp":1702417716738,"Executor ID":"0","Executor Info":{"Host":"10.59.184.244","Total Cores":8,"Log Urls":{"stdout":"http://10.59.184.244:40000/logPage/?appId=app-20231212214826-0000&executorId=0&logType=stdout","stderr":"http://10.59.184.244:40000/logPage/?appId=app-20231212214826-0000&executorId=0&logType=stderr"},"Attributes":{},"Resources":{},"Resource Profile Id":0}} {"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"0","Host":"10.59.184.244","Port":43349},"Maximum Memory":11884245811,"Timestamp":1702417718124,"Maximum Onheap Memory":11884245811,"Maximum Offheap Memory":0} -{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/lib/jvm/java-8-openjdk-amd64/jre","Java Version":"1.8.0_282 (Private Build)","Scala Version":"version 2.12.10"},"Spark Properties":{"spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.scheduler.mode":"FIFO","spark.driver.memory":"30g","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"},"Hadoop Properties":{"hadoop.service.shutdown.timeout":"30s","yarn.resourcemanager.amlauncher.thread-count":"50","yarn.sharedcache.enabled":"false","fs.s3a.connection.maximum":"15","yarn.nodemanager.numa-awareness.numactl.cmd":"/usr/bin/numactl","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","mapreduce.framework.name":"local","yarn.sharedcache.uploader.server.thread-count":"50","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","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","hadoop.registry.secure":"false","hadoop.shell.safely.delete.limit.num.files":"100","dfs.bytes-per-checksum":"512","mapreduce.job.acl-view-job":" ","fs.s3a.s3guard.ddb.background.sleep":"25ms","fs.s3a.retry.limit":"${fs.s3a.attempts.maximum}","mapreduce.jobhistory.loadedjobs.cache.size":"5","fs.s3a.s3guard.ddb.table.create":"false","yarn.nodemanager.amrmproxy.enabled":"false","yarn.timeline-service.entity-group-fs-store.with-user-dir":"false","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.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.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","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","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.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.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","yarn.timeline-service.client.fd-clean-interval-secs":"60","yarn.resourcemanager.nodemanagers.heartbeat-interval-ms":"1000","hadoop.common.configuration.version":"3.0.0","fs.s3a.s3guard.ddb.table.capacity.read":"500","yarn.nodemanager.remote-app-log-dir-suffix":"logs","yarn.nodemanager.windows-container.cpu-limit.enabled":"false","yarn.nodemanager.runtime.linux.docker.privileged-containers.allowed":"false","file.blocksize":"67108864","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","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","fs.har.impl.disable.cache":"true","yarn.webapp.ui2.enable":"false","io.compression.codec.bzip2.library":"system-native","yarn.nodemanager.distributed-scheduling.enabled":"false","mapreduce.shuffle.connection-keep-alive.timeout":"5","yarn.resourcemanager.webapp.https.address":"${yarn.resourcemanager.hostname}:8090","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.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","fs.ftp.transfer.mode":"BLOCK_TRANSFER_MODE","ipc.server.log.slow.rpc":"false","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","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","hadoop.security.credential.clear-text-fallback":"true","yarn.nodemanager.collector-service.thread-count":"5","fs.azure.secure.mode":"false","mapreduce.jobhistory.joblist.cache.size":"20000","fs.ftp.host":"0.0.0.0","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":"false","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","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","yarn.client.load.resource-types.from-server":"false","ha.zookeeper.session-timeout.ms":"10000","tfile.io.chunk.size":"1048576","fs.s3a.s3guard.ddb.table.capacity.write":"100","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","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","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","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","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","mapreduce.job.max.split.locations":"15","yarn.resourcemanager.nm-container-queuing.max-queue-length":"15","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.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","fs.s3a.buffer.dir":"${hadoop.tmp.dir}/s3a","hadoop.ssl.enabled.protocols":"TLSv1,SSLv2Hello,TLSv1.1,TLSv1.2","mapreduce.jobhistory.admin.address":"0.0.0.0:10033","yarn.log-aggregation-status.time-out.ms":"600000","fs.s3a.assumed.role.sts.endpoint.region":"us-west-1","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.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","fs.azure.local.sas.key.mode":"false","ipc.maximum.data.length":"67108864","mapreduce.shuffle.max.threads":"0","yarn.router.pipeline.cache-max-size":"25","yarn.resourcemanager.nm-container-queuing.load-comparator":"QUEUE_LENGTH","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","mapreduce.jobhistory.http.policy":"HTTP_ONLY","yarn.sharedcache.store.in-memory.check-period-mins":"720","hadoop.security.group.mapping.ldap.ssl":"false","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.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.nodemanager.resourcemanager.minimum.version":"NONE","mapreduce.job.speculative.speculative-cap-running-tasks":"0.1","yarn.admin.acl":"*","yarn.nodemanager.recovery.supervised":"false","yarn.sharedcache.admin.thread-count":"1","yarn.resourcemanager.ha.automatic-failover.enabled":"true","mapreduce.reduce.skip.maxgroups":"0","mapreduce.reduce.shuffle.connect.timeout":"180000","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","hadoop.ssl.enabled":"false","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":"10","mapreduce.reduce.shuffle.input.buffer.percent":"0.70","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.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":"2147483647","yarn.resourcemanager.webapp.xfs-filter.xframe-options":"SAMEORIGIN","mapreduce.jobhistory.done-dir":"${yarn.app.mapreduce.am.staging-dir}/history/done","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","yarn.webapp.filter-entity-list-by-user":"false","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}","hadoop.workaround.non.threadsafe.getpwuid":"true","fs.df.interval":"60000","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","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","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.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","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.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","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","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.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","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","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":"${fs.s3a.attempts.maximum}","hadoop.http.authentication.type":"simple","mapreduce.job.cache.limit.max-resources":"0","mapreduce.task.userlog.limit.kb":"0","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","hadoop.user.group.static.mapping.overrides":"dr.who=;","fs.azure.sas.expiry.period":"90d","mapreduce.jobhistory.recovery.store.class":"org.apache.hadoop.mapreduce.v2.hs.HistoryServerFileSystemStateStoreService","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","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","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":"1000ms","seq.io.sort.factor":"100","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/","mapreduce.task.profile":"false","yarn.resourcemanager.fs.state-store.uri":"${hadoop.tmp.dir}/yarn/system/rmstore","mapreduce.jobhistory.always-scan-user-dir":"false","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","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","fs.automatic.close":"true","yarn.nodemanager.hostname":"0.0.0.0","yarn.nodemanager.resource.memory.cgroups.swappiness":"0","ftp.stream-buffer-size":"4096","yarn.fail-fast":"false","yarn.timeline-service.app-aggregation-interval-secs":"15","hadoop.security.group.mapping.ldap.search.filter.user":"(&(objectClass=user)(sAMAccountName={0}))","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","yarn.resourcemanager.delegation.token.max-lifetime":"*********(redacted)","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.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.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.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","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","hadoop.security.dns.log-slow-lookups.enabled":"false","mapreduce.job.committer.setup.cleanup.needed":"true","mapreduce.job.running.reduce.limit":"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","fs.s3a.max.total.tasks":"5","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.nodemanager.health-checker.script.timeout-ms":"1200000","yarn.timeline-service.leveldb-timeline-store.start-time-write-cache-size":"10000","yarn.resourcemanager.resource-profiles.enabled":"false","yarn.timeline-service.hbase-schema.prefix":"prod.","fs.azure.authorization":"false","mapreduce.map.log.level":"INFO","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","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","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","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":"0,1","yarn.timeline-service.client.internal-timers-ttl-secs":"420","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","ipc.client.tcpnodelay":"true","ipc.client.rpc-timeout.ms":"0","yarn.nodemanager.webapp.rest-csrf.methods-to-ignore":"GET,OPTIONS,HEAD","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","fs.s3a.socket.recv.buffer":"8192","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.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","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","mapreduce.shuffle.ssl.enabled":"false","yarn.timeline-service.hbase.coprocessor.app-final-value-retention-milliseconds":"259200000","fs.s3a.committer.staging.abort.pending.uploads":"true","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","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":"100M","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","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","yarn.resourcemanager.ha.enabled":"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.federation.subcluster-resolver.class":"org.apache.hadoop.yarn.server.federation.resolver.DefaultSubClusterResolverImpl","yarn.resourcemanager.zk-state-store.parent-path":"/rmstore","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":"fail","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","fs.AbstractFileSystem.adl.impl":"org.apache.hadoop.fs.adl.Adl","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":"128","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.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.federation.registry.base-dir":"yarnfederation/","mapreduce.job.max.map":"-1","mapreduce.job.local-fs.single-disk-limit.bytes":"-1","mapreduce.job.ubertask.maxreduces":"1","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","yarn.client.nodemanager-connect.retry-interval-ms":"10000","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","yarn.timeline-service.webapp.rest-csrf.enabled":"false"},"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":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Private Build","java.vm.specification.version":"1.8","user.home":"/home/user1","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/amd64","user.dir":"/home/user1/workspace/spark-rapids-another","java.library.path":"/usr/local/cuda/lib64:/usr/java/packages/lib/amd64:/usr/lib/x86_64-linux-gnu/jni:/lib/x86_64-linux-gnu:/usr/lib/x86_64-linux-gnu:/usr/lib/jni:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.282-b08","jetty.git.hash":"e46af88704a893fc12cb0e3bf46e2c7b48a009e7","java.endorsed.dirs":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/endorsed","java.runtime.version":"1.8.0_282-8u282-b08-0ubuntu1~18.04-b08","java.vm.info":"mixed mode","java.ext.dirs":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"OpenJDK Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/resources.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/rt.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/sunrsasign.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jsse.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jce.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/charsets.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jfr.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"4.15.0-130-generic","sun.os.patch.level":"unknown","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","user.language":"en","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.print.PSPrinterJob","java.awt.graphicsenv":"sun.awt.X11GraphicsEnvironment","awt.toolkit":"sun.awt.X11.XToolkit","os.name":"Linux","java.vm.vendor":"Private Build","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"user1","java.vm.name":"OpenJDK 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master local[1] --conf spark.driver.memory=30g --class com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils --jars /home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar /home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar dsAndDf /home/user1/testeventlogDir 100001","java.home":"/usr/lib/jvm/java-8-openjdk-amd64/jre","java.version":"1.8.0_282","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-exec-2.3.7-core.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aopalliance-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/breeze-macros_2.12-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-yarn_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jaxb-api-2.2.11.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-scala_2.12-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okhttp-2.7.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-scheduling-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-jaxrs-json-provider-2.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-storageclass-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-sketch_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/flatbuffers-java-1.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-media-jaxb-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/chill_2.12-0.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-policy-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guava-14.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jline-2.14.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stax-api-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/chill-java-0.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/shims-0.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jdo-api-3.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-autoscaling-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/gson-2.2.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/py4j-0.10.9.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-core_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-memory-netty-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-jaxb-annotations-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-kubernetes_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-admin-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/slf4j-api-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stax2-api-3.1.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-core_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-xdr-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-common-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-apiextensions-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/antlr4-runtime-4.8-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-common-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/antlr-runtime-3.5.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-core-4.1.17.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mesos_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-streaming_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jsp-api-2.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-framework-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/activation-1.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/mesos-1.4.0-shaded-protobuf.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/pyrolite-4.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/libthrift-0.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-ipc-1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-scheduler-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.annotation-api-1.3.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-client-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-crypto-1.1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-util-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-events-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-databind-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/xbean-asm7-shaded-4.15.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-mapreduce-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/core-1.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okio-1.14.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-discovery-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zookeeper-3.4.14.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/woodstox-core-5.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.xml.bind-api-2.3.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-apps-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-service-rpc-3.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mllib-local_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-launcher_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/httpcore-4.4.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jcl-over-slf4j-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-tags_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-jdbc-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-client-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arpack_combined_all-0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/token-provider-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/joda-time-2.10.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-graphx_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-format-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/re2j-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kryo-shaded-4.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jpam-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-hadoop-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/super-csv-2.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/breeze_2.12-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-jackson_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-collections-3.2.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-core-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-beeline-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-ast_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-xml_2.12-1.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-server-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/bonecp-0.8.0.RELEASE.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/slf4j-log4j12-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-shims-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/leveldbjni-all-1.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-reflect-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-serde-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/audience-annotations-0.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-batch-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-coordination-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-common-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-paranamer-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/threeten-extra-1.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-client-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/compress-lzf-1.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javax.inject-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/conf/":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-core-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/accessors-smart-1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-math3-3.4.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/RoaringBitmap-0.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ehcache-3.3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-column-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.activation-api-1.2.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-configuration2-2.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-api-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-recipes-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-network-shuffle_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/xz-1.5.jar":"System Classpath","spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar":"Added By User","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/dropwizard-metrics-hadoop-metrics2-reporter-0.1.2.jar":"System Classpath","spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar":"Added By User","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aircompressor-0.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-graphite-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-jmx-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-server-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/protobuf-java-2.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-util_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-lang3-3.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-memory-core-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-vector-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/generex-1.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/httpclient-4.5.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-client-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-compress-1.20.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-metastore-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-auth-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-scalap_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.ws.rs-api-2.1.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ST4-4.0.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-locator-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-networking-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-extensions-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jaxb-runtime-2.3.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-compiler-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/shapeless_2.12-2.3.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-core-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-pool-1.5.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-metrics-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jta-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ivy-2.4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/paranamer-2.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/istack-commons-runtime-3.0.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-text-1.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-cli-1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-core-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-net-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-utils-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/transaction-api-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/automaton-1.11-8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-parser-combinators_2.12-1.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/cats-kernel_2.12-2.0.0-M4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/nimbus-jose-jwt-4.41.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-util-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-container-servlet-core-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-vector-code-gen-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.validation-api-2.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-jaxrs-base-2.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-common-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-rdbms-4.1.19.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/janino-3.0.16.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/algebra_2.12-2.0.0-M2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/velocity-1.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.servlet-api-4.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-jobclient-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/oro-2.0.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/htrace-core4-4.1.0-incubating.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-mapred-1.8.2-hadoop2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-datatype-jsr310-2.11.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-kvstore_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-repl_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jul-to-slf4j-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-codec-1.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-cli-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-hk2-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guice-4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javassist-3.25.0-GA.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-lang-2.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/macro-compat_2.12-1.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-io-2.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-hive-thriftserver_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-dbcp-1.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-core-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-0.23-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-encoding-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okhttp-3.12.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guice-servlet-4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-hdfs-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-daemon-1.0.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-library-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-api-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jsr305-3.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json-1.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-network-common_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zstd-jni-1.4.8-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/snappy-java-1.1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-macros_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-annotations-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-simplekdc-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-logging-1.1.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stream-2.9.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-httpclient-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-hive_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-jackson-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-format-2.4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-core-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json-smart-2.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javax.jdo-3.2.0-m3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zjsonpatch-0.3.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-pkix-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/opencsv-2.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/machinist_2.12-0.6.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aopalliance-repackaged-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-platform_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/netty-all-4.1.51.Final.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-admissionregistration-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-rbac-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/osgi-resource-locator-1.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-annotations-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-config-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/libfb303-0.9.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-unsafe_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-beanutils-1.9.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-tags_2.12-3.1.1-tests.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-llap-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mllib_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-identity-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/log4j-1.2.17.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-settings-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-collection-compat_2.12-2.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-jvm-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/derby-10.12.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-storage-api-2.7.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/JLargeArrays-1.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-api-jdo-4.2.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/JTransforms-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javolution-5.5.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/geronimo-jcache_1.0_spec-1.0-alpha-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/minlog-1.3.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-container-servlet-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/objenesis-2.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/logging-interceptor-3.12.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-compiler-3.0.16.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-certificates-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jodd-core-3.5.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-sql_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jcip-annotations-1.0-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-asn1-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-json-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.inject-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-server-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-dataformat-yaml-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-server-web-proxy-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-registry-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-catalyst_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/HikariCP-2.5.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-crypto-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/dnsjava-2.1.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/univocity-parsers-2.9.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/lz4-java-1.7.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/snakeyaml-1.24.jar":"System Classpath"}} -{"Event":"SparkListenerApplicationStart","App Name":"Rapids Spark Profiling Tool Unit Tests","App ID":"local-1622043423018","Timestamp":1622043422343,"User":"user1"} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/lib/jvm/java-8-openjdk-amd64/jre","Java Version":"1.8.0_282 (Private Build)","Scala Version":"version 2.12.10"},"Spark Properties":{"spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.scheduler.mode":"FIFO","spark.driver.memory":"30g","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423011"},"Hadoop Properties":{"hadoop.service.shutdown.timeout":"30s","yarn.resourcemanager.amlauncher.thread-count":"50","yarn.sharedcache.enabled":"false","fs.s3a.connection.maximum":"15","yarn.nodemanager.numa-awareness.numactl.cmd":"/usr/bin/numactl","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","mapreduce.framework.name":"local","yarn.sharedcache.uploader.server.thread-count":"50","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","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","hadoop.registry.secure":"false","hadoop.shell.safely.delete.limit.num.files":"100","dfs.bytes-per-checksum":"512","mapreduce.job.acl-view-job":" ","fs.s3a.s3guard.ddb.background.sleep":"25ms","fs.s3a.retry.limit":"${fs.s3a.attempts.maximum}","mapreduce.jobhistory.loadedjobs.cache.size":"5","fs.s3a.s3guard.ddb.table.create":"false","yarn.nodemanager.amrmproxy.enabled":"false","yarn.timeline-service.entity-group-fs-store.with-user-dir":"false","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.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.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","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","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.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.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","yarn.timeline-service.client.fd-clean-interval-secs":"60","yarn.resourcemanager.nodemanagers.heartbeat-interval-ms":"1000","hadoop.common.configuration.version":"3.0.0","fs.s3a.s3guard.ddb.table.capacity.read":"500","yarn.nodemanager.remote-app-log-dir-suffix":"logs","yarn.nodemanager.windows-container.cpu-limit.enabled":"false","yarn.nodemanager.runtime.linux.docker.privileged-containers.allowed":"false","file.blocksize":"67108864","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","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","fs.har.impl.disable.cache":"true","yarn.webapp.ui2.enable":"false","io.compression.codec.bzip2.library":"system-native","yarn.nodemanager.distributed-scheduling.enabled":"false","mapreduce.shuffle.connection-keep-alive.timeout":"5","yarn.resourcemanager.webapp.https.address":"${yarn.resourcemanager.hostname}:8090","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.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","fs.ftp.transfer.mode":"BLOCK_TRANSFER_MODE","ipc.server.log.slow.rpc":"false","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","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","hadoop.security.credential.clear-text-fallback":"true","yarn.nodemanager.collector-service.thread-count":"5","fs.azure.secure.mode":"false","mapreduce.jobhistory.joblist.cache.size":"20000","fs.ftp.host":"0.0.0.0","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":"false","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","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","yarn.client.load.resource-types.from-server":"false","ha.zookeeper.session-timeout.ms":"10000","tfile.io.chunk.size":"1048576","fs.s3a.s3guard.ddb.table.capacity.write":"100","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","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","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","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","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","mapreduce.job.max.split.locations":"15","yarn.resourcemanager.nm-container-queuing.max-queue-length":"15","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.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","fs.s3a.buffer.dir":"${hadoop.tmp.dir}/s3a","hadoop.ssl.enabled.protocols":"TLSv1,SSLv2Hello,TLSv1.1,TLSv1.2","mapreduce.jobhistory.admin.address":"0.0.0.0:10033","yarn.log-aggregation-status.time-out.ms":"600000","fs.s3a.assumed.role.sts.endpoint.region":"us-west-1","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.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","fs.azure.local.sas.key.mode":"false","ipc.maximum.data.length":"67108864","mapreduce.shuffle.max.threads":"0","yarn.router.pipeline.cache-max-size":"25","yarn.resourcemanager.nm-container-queuing.load-comparator":"QUEUE_LENGTH","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","mapreduce.jobhistory.http.policy":"HTTP_ONLY","yarn.sharedcache.store.in-memory.check-period-mins":"720","hadoop.security.group.mapping.ldap.ssl":"false","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.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.nodemanager.resourcemanager.minimum.version":"NONE","mapreduce.job.speculative.speculative-cap-running-tasks":"0.1","yarn.admin.acl":"*","yarn.nodemanager.recovery.supervised":"false","yarn.sharedcache.admin.thread-count":"1","yarn.resourcemanager.ha.automatic-failover.enabled":"true","mapreduce.reduce.skip.maxgroups":"0","mapreduce.reduce.shuffle.connect.timeout":"180000","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","hadoop.ssl.enabled":"false","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":"10","mapreduce.reduce.shuffle.input.buffer.percent":"0.70","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.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":"2147483647","yarn.resourcemanager.webapp.xfs-filter.xframe-options":"SAMEORIGIN","mapreduce.jobhistory.done-dir":"${yarn.app.mapreduce.am.staging-dir}/history/done","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","yarn.webapp.filter-entity-list-by-user":"false","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}","hadoop.workaround.non.threadsafe.getpwuid":"true","fs.df.interval":"60000","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","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","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.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","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.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","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","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.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","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","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":"${fs.s3a.attempts.maximum}","hadoop.http.authentication.type":"simple","mapreduce.job.cache.limit.max-resources":"0","mapreduce.task.userlog.limit.kb":"0","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","hadoop.user.group.static.mapping.overrides":"dr.who=;","fs.azure.sas.expiry.period":"90d","mapreduce.jobhistory.recovery.store.class":"org.apache.hadoop.mapreduce.v2.hs.HistoryServerFileSystemStateStoreService","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","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","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":"1000ms","seq.io.sort.factor":"100","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/","mapreduce.task.profile":"false","yarn.resourcemanager.fs.state-store.uri":"${hadoop.tmp.dir}/yarn/system/rmstore","mapreduce.jobhistory.always-scan-user-dir":"false","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","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","fs.automatic.close":"true","yarn.nodemanager.hostname":"0.0.0.0","yarn.nodemanager.resource.memory.cgroups.swappiness":"0","ftp.stream-buffer-size":"4096","yarn.fail-fast":"false","yarn.timeline-service.app-aggregation-interval-secs":"15","hadoop.security.group.mapping.ldap.search.filter.user":"(&(objectClass=user)(sAMAccountName={0}))","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","yarn.resourcemanager.delegation.token.max-lifetime":"*********(redacted)","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.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.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.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","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","hadoop.security.dns.log-slow-lookups.enabled":"false","mapreduce.job.committer.setup.cleanup.needed":"true","mapreduce.job.running.reduce.limit":"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","fs.s3a.max.total.tasks":"5","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.nodemanager.health-checker.script.timeout-ms":"1200000","yarn.timeline-service.leveldb-timeline-store.start-time-write-cache-size":"10000","yarn.resourcemanager.resource-profiles.enabled":"false","yarn.timeline-service.hbase-schema.prefix":"prod.","fs.azure.authorization":"false","mapreduce.map.log.level":"INFO","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","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","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","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":"0,1","yarn.timeline-service.client.internal-timers-ttl-secs":"420","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","ipc.client.tcpnodelay":"true","ipc.client.rpc-timeout.ms":"0","yarn.nodemanager.webapp.rest-csrf.methods-to-ignore":"GET,OPTIONS,HEAD","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","fs.s3a.socket.recv.buffer":"8192","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.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","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","mapreduce.shuffle.ssl.enabled":"false","yarn.timeline-service.hbase.coprocessor.app-final-value-retention-milliseconds":"259200000","fs.s3a.committer.staging.abort.pending.uploads":"true","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","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":"100M","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","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","yarn.resourcemanager.ha.enabled":"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.federation.subcluster-resolver.class":"org.apache.hadoop.yarn.server.federation.resolver.DefaultSubClusterResolverImpl","yarn.resourcemanager.zk-state-store.parent-path":"/rmstore","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":"fail","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","fs.AbstractFileSystem.adl.impl":"org.apache.hadoop.fs.adl.Adl","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":"128","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.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.federation.registry.base-dir":"yarnfederation/","mapreduce.job.max.map":"-1","mapreduce.job.local-fs.single-disk-limit.bytes":"-1","mapreduce.job.ubertask.maxreduces":"1","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","yarn.client.nodemanager-connect.retry-interval-ms":"10000","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","yarn.timeline-service.webapp.rest-csrf.enabled":"false"},"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":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Private Build","java.vm.specification.version":"1.8","user.home":"/home/user1","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/amd64","user.dir":"/home/user1/workspace/spark-rapids-another","java.library.path":"/usr/local/cuda/lib64:/usr/java/packages/lib/amd64:/usr/lib/x86_64-linux-gnu/jni:/lib/x86_64-linux-gnu:/usr/lib/x86_64-linux-gnu:/usr/lib/jni:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.282-b08","jetty.git.hash":"e46af88704a893fc12cb0e3bf46e2c7b48a009e7","java.endorsed.dirs":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/endorsed","java.runtime.version":"1.8.0_282-8u282-b08-0ubuntu1~18.04-b08","java.vm.info":"mixed mode","java.ext.dirs":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"OpenJDK Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/resources.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/rt.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/sunrsasign.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jsse.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jce.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/charsets.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jfr.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"4.15.0-130-generic","sun.os.patch.level":"unknown","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","user.language":"en","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.print.PSPrinterJob","java.awt.graphicsenv":"sun.awt.X11GraphicsEnvironment","awt.toolkit":"sun.awt.X11.XToolkit","os.name":"Linux","java.vm.vendor":"Private Build","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"user1","java.vm.name":"OpenJDK 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master local[1] --conf spark.driver.memory=30g --class com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils --jars /home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar /home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar dsAndDf /home/user1/testeventlogDir 100001","java.home":"/usr/lib/jvm/java-8-openjdk-amd64/jre","java.version":"1.8.0_282","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-exec-2.3.7-core.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aopalliance-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/breeze-macros_2.12-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-yarn_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jaxb-api-2.2.11.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-scala_2.12-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okhttp-2.7.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-scheduling-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-jaxrs-json-provider-2.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-storageclass-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-sketch_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/flatbuffers-java-1.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-media-jaxb-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/chill_2.12-0.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-policy-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guava-14.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jline-2.14.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stax-api-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/chill-java-0.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/shims-0.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jdo-api-3.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-autoscaling-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/gson-2.2.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/py4j-0.10.9.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-core_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-memory-netty-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-jaxb-annotations-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-kubernetes_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-admin-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/slf4j-api-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stax2-api-3.1.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-core_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-xdr-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-common-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-apiextensions-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/antlr4-runtime-4.8-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-common-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/antlr-runtime-3.5.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-core-4.1.17.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mesos_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-streaming_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jsp-api-2.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-framework-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/activation-1.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/mesos-1.4.0-shaded-protobuf.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/pyrolite-4.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/libthrift-0.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-ipc-1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-scheduler-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.annotation-api-1.3.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-client-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-crypto-1.1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-util-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-events-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-databind-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/xbean-asm7-shaded-4.15.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-mapreduce-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/core-1.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okio-1.14.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-discovery-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zookeeper-3.4.14.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/woodstox-core-5.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.xml.bind-api-2.3.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-apps-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-service-rpc-3.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mllib-local_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-launcher_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/httpcore-4.4.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jcl-over-slf4j-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-tags_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-jdbc-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-client-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arpack_combined_all-0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/token-provider-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/joda-time-2.10.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-graphx_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-format-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/re2j-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kryo-shaded-4.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jpam-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-hadoop-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/super-csv-2.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/breeze_2.12-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-jackson_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-collections-3.2.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-core-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-beeline-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-ast_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-xml_2.12-1.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-server-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/bonecp-0.8.0.RELEASE.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/slf4j-log4j12-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-shims-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/leveldbjni-all-1.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-reflect-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-serde-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/audience-annotations-0.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-batch-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-coordination-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-common-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-paranamer-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/threeten-extra-1.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-client-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/compress-lzf-1.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javax.inject-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/conf/":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-core-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/accessors-smart-1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-math3-3.4.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/RoaringBitmap-0.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ehcache-3.3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-column-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.activation-api-1.2.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-configuration2-2.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-api-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-recipes-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-network-shuffle_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/xz-1.5.jar":"System Classpath","spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar":"Added By User","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/dropwizard-metrics-hadoop-metrics2-reporter-0.1.2.jar":"System Classpath","spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar":"Added By User","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aircompressor-0.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-graphite-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-jmx-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-server-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/protobuf-java-2.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-util_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-lang3-3.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-memory-core-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-vector-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/generex-1.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/httpclient-4.5.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-client-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-compress-1.20.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-metastore-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-auth-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-scalap_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.ws.rs-api-2.1.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ST4-4.0.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-locator-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-networking-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-extensions-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jaxb-runtime-2.3.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-compiler-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/shapeless_2.12-2.3.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-core-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-pool-1.5.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-metrics-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jta-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ivy-2.4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/paranamer-2.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/istack-commons-runtime-3.0.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-text-1.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-cli-1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-core-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-net-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-utils-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/transaction-api-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/automaton-1.11-8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-parser-combinators_2.12-1.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/cats-kernel_2.12-2.0.0-M4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/nimbus-jose-jwt-4.41.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-util-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-container-servlet-core-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-vector-code-gen-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.validation-api-2.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-jaxrs-base-2.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-common-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-rdbms-4.1.19.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/janino-3.0.16.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/algebra_2.12-2.0.0-M2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/velocity-1.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.servlet-api-4.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-jobclient-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/oro-2.0.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/htrace-core4-4.1.0-incubating.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-mapred-1.8.2-hadoop2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-datatype-jsr310-2.11.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-kvstore_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-repl_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jul-to-slf4j-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-codec-1.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-cli-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-hk2-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guice-4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javassist-3.25.0-GA.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-lang-2.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/macro-compat_2.12-1.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-io-2.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-hive-thriftserver_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-dbcp-1.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-core-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-0.23-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-encoding-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okhttp-3.12.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guice-servlet-4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-hdfs-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-daemon-1.0.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-library-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-api-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jsr305-3.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json-1.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-network-common_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zstd-jni-1.4.8-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/snappy-java-1.1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-macros_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-annotations-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-simplekdc-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-logging-1.1.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stream-2.9.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-httpclient-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-hive_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-jackson-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-format-2.4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-core-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json-smart-2.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javax.jdo-3.2.0-m3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zjsonpatch-0.3.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-pkix-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/opencsv-2.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/machinist_2.12-0.6.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aopalliance-repackaged-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-platform_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/netty-all-4.1.51.Final.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-admissionregistration-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-rbac-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/osgi-resource-locator-1.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-annotations-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-config-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/libfb303-0.9.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-unsafe_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-beanutils-1.9.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-tags_2.12-3.1.1-tests.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-llap-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mllib_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-identity-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/log4j-1.2.17.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-settings-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-collection-compat_2.12-2.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-jvm-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/derby-10.12.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-storage-api-2.7.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/JLargeArrays-1.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-api-jdo-4.2.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/JTransforms-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javolution-5.5.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/geronimo-jcache_1.0_spec-1.0-alpha-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/minlog-1.3.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-container-servlet-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/objenesis-2.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/logging-interceptor-3.12.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-compiler-3.0.16.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-certificates-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jodd-core-3.5.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-sql_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jcip-annotations-1.0-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-asn1-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-json-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.inject-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-server-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-dataformat-yaml-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-server-web-proxy-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-registry-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-catalyst_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/HikariCP-2.5.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-crypto-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/dnsjava-2.1.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/univocity-parsers-2.9.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/lz4-java-1.7.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/snakeyaml-1.24.jar":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Rapids Spark Profiling Tool Unit Tests","App ID":"local-1622043423011","Timestamp":1622043422343,"User":"user1"} {"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":0,"description":"json at QualificationInfoUtils.scala:76","details":"org.apache.spark.sql.DataFrameWriter.json(DataFrameWriter.scala:853)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1(QualificationInfoUtils.scala:76)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1$adapted(QualificationInfoUtils.scala:73)\norg.apache.spark.rapids.TestUtils$.withTempPath(TestUtils.scala:31)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genDatasetEventLog(QualificationInfoUtils.scala:73)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:185)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nExecute InsertIntoHadoopFsRelationCommand (8)\n+- * SerializeFromObject (7)\n +- * MapElements (6)\n +- * Filter (5)\n +- * DeserializeToObject (4)\n +- * Project (3)\n +- * SerializeFromObject (2)\n +- Scan (1)\n\n\n(1) Scan\nOutput [1]: [obj#3]\nArguments: obj#3: scala.Tuple3, ParallelCollectionRDD[0] at parallelize at QualificationInfoUtils.scala:64\n\n(2) SerializeFromObject [codegen id : 1]\nInput [1]: [obj#3]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._1, true, false) AS _1#4, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._2, true, false) AS _2#5, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._3 AS _3#6]\n\n(3) Project [codegen id : 1]\nOutput [3]: [_1#4 AS name#11, _2#5 AS friend#12, _3#6 AS age#13]\nInput [3]: [_1#4, _2#5, _3#6]\n\n(4) DeserializeToObject [codegen id : 1]\nInput [3]: [name#11, friend#12, age#13]\nArguments: newInstance(class com.nvidia.spark.rapids.tool.profiling.RapidsFriends), obj#30: com.nvidia.spark.rapids.tool.profiling.RapidsFriends\n\n(5) Filter [codegen id : 1]\nInput [1]: [obj#30]\nCondition : com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$$$Lambda$1503/443638755@31f31b20.apply\n\n(6) MapElements [codegen id : 1]\nInput [1]: [obj#30]\nArguments: com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$$$Lambda$1505/1108374296@6723cce7, obj#31: scala.Tuple2\n\n(7) SerializeFromObject [codegen id : 1]\nInput [1]: [obj#31]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1, true, false) AS _1#32, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._2 AS _2#33]\n\n(8) Execute InsertIntoHadoopFsRelationCommand\nInput [2]: [_1#32, _2#33]\nArguments: file:/tmp/spark-4a9e661e-fc37-4f5f-8bbc-1647dbaebbaf, false, JSON, Map(path -> /tmp/spark-4a9e661e-fc37-4f5f-8bbc-1647dbaebbaf), ErrorIfExists, [_1, _2]\n\n","sparkPlanInfo":{"nodeName":"Execute InsertIntoHadoopFsRelationCommand","simpleString":"Execute InsertIntoHadoopFsRelationCommand file:/tmp/spark-4a9e661e-fc37-4f5f-8bbc-1647dbaebbaf, false, JSON, Map(path -> /tmp/spark-4a9e661e-fc37-4f5f-8bbc-1647dbaebbaf), ErrorIfExists, [_1, _2]","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1, true, false) AS _1#32, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._2 AS _2#33]","children":[{"nodeName":"MapElements","simpleString":"MapElements com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$$$Lambda$1505/1108374296@6723cce7, obj#31: scala.Tuple2","children":[{"nodeName":"Filter","simpleString":"Filter com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$$$Lambda$1503/443638755@31f31b20.apply","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject newInstance(class com.nvidia.spark.rapids.tool.profiling.RapidsFriends), obj#30: com.nvidia.spark.rapids.tool.profiling.RapidsFriends","children":[{"nodeName":"Project","simpleString":"Project [_1#4 AS name#11, _2#5 AS friend#12, _3#6 AS age#13]","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._1, true, false) AS _1#4, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._2, true, false) AS _2#5, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._3 AS _3#6]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Scan","simpleString":"Scan[obj#3]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":6,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":5,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":4,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of written files","accumulatorId":0,"metricType":"sum"},{"name":"written output","accumulatorId":1,"metricType":"size"},{"name":"number of output rows","accumulatorId":2,"metricType":"sum"},{"name":"number of dynamic part","accumulatorId":3,"metricType":"sum"}]},"time":1622043425909} -{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1622043426367,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"json at QualificationInfoUtils.scala:76","Number of Tasks":12,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"Scan\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at QualificationInfoUtils.scala:64","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameWriter.json(DataFrameWriter.scala:853)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1(QualificationInfoUtils.scala:76)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1$adapted(QualificationInfoUtils.scala:73)\norg.apache.spark.rapids.TestUtils$.withTempPath(TestUtils.scala:31)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genDatasetEventLog(QualificationInfoUtils.scala:73)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:185)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[0],"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"1\",\"name\":\"Execute InsertIntoHadoopFsRelationCommand\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"0","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"json at QualificationInfoUtils.scala:76","Number of Tasks":12,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"Scan\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at QualificationInfoUtils.scala:64","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameWriter.json(DataFrameWriter.scala:853)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1(QualificationInfoUtils.scala:76)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1$adapted(QualificationInfoUtils.scala:73)\norg.apache.spark.rapids.TestUtils$.withTempPath(TestUtils.scala:31)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genDatasetEventLog(QualificationInfoUtils.scala:73)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:185)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043426387,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"1\",\"name\":\"Execute InsertIntoHadoopFsRelationCommand\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"0","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"}} +{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1622043426367,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"json at QualificationInfoUtils.scala:76","Number of Tasks":12,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"Scan\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at QualificationInfoUtils.scala:64","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameWriter.json(DataFrameWriter.scala:853)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1(QualificationInfoUtils.scala:76)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1$adapted(QualificationInfoUtils.scala:73)\norg.apache.spark.rapids.TestUtils$.withTempPath(TestUtils.scala:31)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genDatasetEventLog(QualificationInfoUtils.scala:73)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:185)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[0],"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"1\",\"name\":\"Execute InsertIntoHadoopFsRelationCommand\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"0","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423011"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"json at QualificationInfoUtils.scala:76","Number of Tasks":12,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"Scan\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at QualificationInfoUtils.scala:64","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameWriter.json(DataFrameWriter.scala:853)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1(QualificationInfoUtils.scala:76)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1$adapted(QualificationInfoUtils.scala:73)\norg.apache.spark.rapids.TestUtils$.withTempPath(TestUtils.scala:31)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genDatasetEventLog(QualificationInfoUtils.scala:73)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:185)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043426387,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"1\",\"name\":\"Execute InsertIntoHadoopFsRelationCommand\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"0","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423011"}} {"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1622043426546,"Executor ID":"driver","Host":"10.10.10.100","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} {"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1622043426567,"Executor ID":"driver","Host":"10.10.10.100","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} {"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1622043426568,"Executor ID":"driver","Host":"10.10.10.100","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} @@ -39,9 +39,9 @@ {"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":0,"accumUpdates":[[0,12],[1,16560],[2,720],[3,0]]} {"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":0,"time":1622043427215} {"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":1,"description":"count at QualificationInfoUtils.scala:94","details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\n* HashAggregate (15)\n+- Exchange (14)\n +- * HashAggregate (13)\n +- * Project (12)\n +- * SortMergeJoin Inner (11)\n :- * Sort (5)\n : +- Exchange (4)\n : +- * Project (3)\n : +- * SerializeFromObject (2)\n : +- Scan (1)\n +- * Sort (10)\n +- Exchange (9)\n +- * Project (8)\n +- * SerializeFromObject (7)\n +- Scan (6)\n\n\n(1) Scan\nOutput [1]: [obj#40]\nArguments: obj#40: int, ParallelCollectionRDD[5] at makeRDD at QualificationInfoUtils.scala:92\n\n(2) SerializeFromObject [codegen id : 1]\nInput [1]: [obj#40]\nArguments: [input[0, int, false] AS value#41]\n\n(3) Project [codegen id : 1]\nOutput [1]: [value#41 AS a#51]\nInput [1]: [value#41]\n\n(4) Exchange\nInput [1]: [a#51]\nArguments: hashpartitioning(a#51, 200), ENSURE_REQUIREMENTS, [id=#93]\n\n(5) Sort [codegen id : 2]\nInput [1]: [a#51]\nArguments: [a#51 ASC NULLS FIRST], false, 0\n\n(6) Scan\nOutput [1]: [obj#46]\nArguments: obj#46: int, ParallelCollectionRDD[6] at makeRDD at QualificationInfoUtils.scala:93\n\n(7) SerializeFromObject [codegen id : 3]\nInput [1]: [obj#46]\nArguments: [input[0, int, false] AS value#47]\n\n(8) Project [codegen id : 3]\nOutput [1]: [value#47 AS b#53]\nInput [1]: [value#47]\n\n(9) Exchange\nInput [1]: [b#53]\nArguments: hashpartitioning(b#53, 200), ENSURE_REQUIREMENTS, [id=#102]\n\n(10) Sort [codegen id : 4]\nInput [1]: [b#53]\nArguments: [b#53 ASC NULLS FIRST], false, 0\n\n(11) SortMergeJoin [codegen id : 5]\nLeft keys [1]: [a#51]\nRight keys [1]: [b#53]\nJoin condition: None\n\n(12) Project [codegen id : 5]\nOutput: []\nInput [2]: [a#51, b#53]\n\n(13) HashAggregate [codegen id : 5]\nInput: []\nKeys: []\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count#64L]\nResults [1]: [count#65L]\n\n(14) Exchange\nInput [1]: [count#65L]\nArguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#111]\n\n(15) HashAggregate [codegen id : 6]\nInput [1]: [count#65L]\nKeys: []\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#61L]\nResults [1]: [count(1)#61L AS count#62L]\n\n","sparkPlanInfo":{"nodeName":"WholeStageCodegen (6)","simpleString":"WholeStageCodegen (6)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Exchange","simpleString":"Exchange SinglePartition, ENSURE_REQUIREMENTS, [id=#111]","children":[{"nodeName":"WholeStageCodegen (5)","simpleString":"WholeStageCodegen (5)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[], functions=[partial_count(1)])","children":[{"nodeName":"Project","simpleString":"Project","children":[{"nodeName":"SortMergeJoin","simpleString":"SortMergeJoin [a#51], [b#53], Inner","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"Sort","simpleString":"Sort [a#51 ASC NULLS FIRST], false, 0","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(a#51, 200), ENSURE_REQUIREMENTS, [id=#93]","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [value#41 AS a#51]","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [input[0, int, false] AS value#41]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Scan","simpleString":"Scan[obj#40]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":116,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":115,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":74,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":75,"metricType":"nsTiming"},{"name":"records read","accumulatorId":72,"metricType":"sum"},{"name":"local bytes read","accumulatorId":70,"metricType":"size"},{"name":"fetch wait time","accumulatorId":71,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":68,"metricType":"size"},{"name":"local blocks read","accumulatorId":67,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":66,"metricType":"sum"},{"name":"data size","accumulatorId":65,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":69,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":73,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"sort time","accumulatorId":112,"metricType":"timing"},{"name":"peak memory","accumulatorId":113,"metricType":"size"},{"name":"spill size","accumulatorId":114,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":111,"metricType":"timing"}]}],"metadata":{},"metrics":[]},{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"Sort","simpleString":"Sort [b#53 ASC NULLS FIRST], false, 0","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(b#53, 200), ENSURE_REQUIREMENTS, [id=#102]","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"Project","simpleString":"Project [value#47 AS b#53]","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [input[0, int, false] AS value#47]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Scan","simpleString":"Scan[obj#46]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":122,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":121,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":85,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":86,"metricType":"nsTiming"},{"name":"records read","accumulatorId":83,"metricType":"sum"},{"name":"local bytes read","accumulatorId":81,"metricType":"size"},{"name":"fetch wait time","accumulatorId":82,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":79,"metricType":"size"},{"name":"local blocks read","accumulatorId":78,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":77,"metricType":"sum"},{"name":"data size","accumulatorId":76,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":80,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":84,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"sort time","accumulatorId":118,"metricType":"timing"},{"name":"peak memory","accumulatorId":119,"metricType":"size"},{"name":"spill size","accumulatorId":120,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":117,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":110,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":107,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":108,"metricType":"timing"},{"name":"peak memory","accumulatorId":106,"metricType":"size"},{"name":"number of output rows","accumulatorId":105,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":109,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":104,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":96,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":97,"metricType":"nsTiming"},{"name":"records read","accumulatorId":94,"metricType":"sum"},{"name":"local bytes read","accumulatorId":92,"metricType":"size"},{"name":"fetch wait time","accumulatorId":93,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":90,"metricType":"size"},{"name":"local blocks read","accumulatorId":89,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":88,"metricType":"sum"},{"name":"data size","accumulatorId":87,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":91,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":95,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":101,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":102,"metricType":"timing"},{"name":"peak memory","accumulatorId":100,"metricType":"size"},{"name":"number of output rows","accumulatorId":99,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":103,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":98,"metricType":"timing"}]},"time":1622043427414} -{"Event":"SparkListenerJobStart","Job ID":1,"Submission Time":1622043427637,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"43\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"15\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:93","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"39\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"14\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:92","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"29\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":200,"RDD Info":[{"RDD ID":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[18],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[14],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"20\",\"name\":\"WholeStageCodegen (5)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"35\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"25\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"20\",\"name\":\"WholeStageCodegen (5)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[11,16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[1,2],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":1,"RDD Info":[{"RDD ID":22,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"44\",\"name\":\"mapPartitionsInternal\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":21,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"16\",\"name\":\"WholeStageCodegen (6)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"19\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[3],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[1,2,3,4],"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"43\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"15\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:93","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"39\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043427644,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"14\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:92","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"29\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043427656,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"}} +{"Event":"SparkListenerJobStart","Job ID":1,"Submission Time":1622043427637,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"43\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"15\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:93","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"39\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"14\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:92","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"29\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":200,"RDD Info":[{"RDD ID":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[18],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[14],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"20\",\"name\":\"WholeStageCodegen (5)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"35\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"25\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"20\",\"name\":\"WholeStageCodegen (5)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[11,16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[1,2],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":1,"RDD Info":[{"RDD ID":22,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"44\",\"name\":\"mapPartitionsInternal\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":21,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"16\",\"name\":\"WholeStageCodegen (6)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"19\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[3],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[1,2,3,4],"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423011"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"43\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"15\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:93","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"39\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043427644,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423011"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"14\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:92","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"29\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043427656,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423011"}} {"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":0,"Attempt":0,"Launch Time":1622043427655,"Executor ID":"driver","Host":"10.10.10.100","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} {"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":1,"Attempt":0,"Launch Time":1622043427661,"Executor ID":"driver","Host":"10.10.10.100","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} {"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":2,"Attempt":0,"Launch Time":1622043427661,"Executor ID":"driver","Host":"10.10.10.100","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} diff --git a/core/src/test/resources/spark-events-qualification/cluster_information/eventlog_3nodes_12cores_exec_removed b/core/src/test/resources/spark-events-qualification/cluster_information/eventlog_3nodes_12cores_exec_removed index 2f18a96c8..4f81bae3d 100644 --- a/core/src/test/resources/spark-events-qualification/cluster_information/eventlog_3nodes_12cores_exec_removed +++ b/core/src/test/resources/spark-events-qualification/cluster_information/eventlog_3nodes_12cores_exec_removed @@ -5,11 +5,11 @@ {"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"10.59.184.228","Port":46767},"Maximum Memory":11884245811,"Timestamp":1702417716594,"Maximum Onheap Memory":11884245811,"Maximum Offheap Memory":0} {"Event":"SparkListenerExecutorAdded","Timestamp":1702417716738,"Executor ID":"0","Executor Info":{"Host":"10.59.184.244","Total Cores":12,"Log Urls":{"stdout":"http://10.59.184.244:40000/logPage/?appId=app-20231212214826-0000&executorId=0&logType=stdout","stderr":"http://10.59.184.244:40000/logPage/?appId=app-20231212214826-0000&executorId=0&logType=stderr"},"Attributes":{},"Resources":{},"Resource Profile Id":0}} {"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"0","Host":"10.59.184.244","Port":43349},"Maximum Memory":11884245811,"Timestamp":1702417718124,"Maximum Onheap Memory":11884245811,"Maximum Offheap Memory":0} -{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/lib/jvm/java-8-openjdk-amd64/jre","Java Version":"1.8.0_282 (Private Build)","Scala Version":"version 2.12.10"},"Spark Properties":{"spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.scheduler.mode":"FIFO","spark.driver.memory":"30g","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"},"Hadoop Properties":{"hadoop.service.shutdown.timeout":"30s","yarn.resourcemanager.amlauncher.thread-count":"50","yarn.sharedcache.enabled":"false","fs.s3a.connection.maximum":"15","yarn.nodemanager.numa-awareness.numactl.cmd":"/usr/bin/numactl","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","mapreduce.framework.name":"local","yarn.sharedcache.uploader.server.thread-count":"50","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","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","hadoop.registry.secure":"false","hadoop.shell.safely.delete.limit.num.files":"100","dfs.bytes-per-checksum":"512","mapreduce.job.acl-view-job":" ","fs.s3a.s3guard.ddb.background.sleep":"25ms","fs.s3a.retry.limit":"${fs.s3a.attempts.maximum}","mapreduce.jobhistory.loadedjobs.cache.size":"5","fs.s3a.s3guard.ddb.table.create":"false","yarn.nodemanager.amrmproxy.enabled":"false","yarn.timeline-service.entity-group-fs-store.with-user-dir":"false","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.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.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","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","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.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.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","yarn.timeline-service.client.fd-clean-interval-secs":"60","yarn.resourcemanager.nodemanagers.heartbeat-interval-ms":"1000","hadoop.common.configuration.version":"3.0.0","fs.s3a.s3guard.ddb.table.capacity.read":"500","yarn.nodemanager.remote-app-log-dir-suffix":"logs","yarn.nodemanager.windows-container.cpu-limit.enabled":"false","yarn.nodemanager.runtime.linux.docker.privileged-containers.allowed":"false","file.blocksize":"67108864","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","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","fs.har.impl.disable.cache":"true","yarn.webapp.ui2.enable":"false","io.compression.codec.bzip2.library":"system-native","yarn.nodemanager.distributed-scheduling.enabled":"false","mapreduce.shuffle.connection-keep-alive.timeout":"5","yarn.resourcemanager.webapp.https.address":"${yarn.resourcemanager.hostname}:8090","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.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","fs.ftp.transfer.mode":"BLOCK_TRANSFER_MODE","ipc.server.log.slow.rpc":"false","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","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","hadoop.security.credential.clear-text-fallback":"true","yarn.nodemanager.collector-service.thread-count":"5","fs.azure.secure.mode":"false","mapreduce.jobhistory.joblist.cache.size":"20000","fs.ftp.host":"0.0.0.0","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":"false","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","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","yarn.client.load.resource-types.from-server":"false","ha.zookeeper.session-timeout.ms":"10000","tfile.io.chunk.size":"1048576","fs.s3a.s3guard.ddb.table.capacity.write":"100","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","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","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","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","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","mapreduce.job.max.split.locations":"15","yarn.resourcemanager.nm-container-queuing.max-queue-length":"15","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.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","fs.s3a.buffer.dir":"${hadoop.tmp.dir}/s3a","hadoop.ssl.enabled.protocols":"TLSv1,SSLv2Hello,TLSv1.1,TLSv1.2","mapreduce.jobhistory.admin.address":"0.0.0.0:10033","yarn.log-aggregation-status.time-out.ms":"600000","fs.s3a.assumed.role.sts.endpoint.region":"us-west-1","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.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","fs.azure.local.sas.key.mode":"false","ipc.maximum.data.length":"67108864","mapreduce.shuffle.max.threads":"0","yarn.router.pipeline.cache-max-size":"25","yarn.resourcemanager.nm-container-queuing.load-comparator":"QUEUE_LENGTH","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","mapreduce.jobhistory.http.policy":"HTTP_ONLY","yarn.sharedcache.store.in-memory.check-period-mins":"720","hadoop.security.group.mapping.ldap.ssl":"false","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.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.nodemanager.resourcemanager.minimum.version":"NONE","mapreduce.job.speculative.speculative-cap-running-tasks":"0.1","yarn.admin.acl":"*","yarn.nodemanager.recovery.supervised":"false","yarn.sharedcache.admin.thread-count":"1","yarn.resourcemanager.ha.automatic-failover.enabled":"true","mapreduce.reduce.skip.maxgroups":"0","mapreduce.reduce.shuffle.connect.timeout":"180000","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","hadoop.ssl.enabled":"false","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":"10","mapreduce.reduce.shuffle.input.buffer.percent":"0.70","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.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":"2147483647","yarn.resourcemanager.webapp.xfs-filter.xframe-options":"SAMEORIGIN","mapreduce.jobhistory.done-dir":"${yarn.app.mapreduce.am.staging-dir}/history/done","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","yarn.webapp.filter-entity-list-by-user":"false","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}","hadoop.workaround.non.threadsafe.getpwuid":"true","fs.df.interval":"60000","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","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","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.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","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.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","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","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.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","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","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":"${fs.s3a.attempts.maximum}","hadoop.http.authentication.type":"simple","mapreduce.job.cache.limit.max-resources":"0","mapreduce.task.userlog.limit.kb":"0","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","hadoop.user.group.static.mapping.overrides":"dr.who=;","fs.azure.sas.expiry.period":"90d","mapreduce.jobhistory.recovery.store.class":"org.apache.hadoop.mapreduce.v2.hs.HistoryServerFileSystemStateStoreService","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","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","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":"1000ms","seq.io.sort.factor":"100","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/","mapreduce.task.profile":"false","yarn.resourcemanager.fs.state-store.uri":"${hadoop.tmp.dir}/yarn/system/rmstore","mapreduce.jobhistory.always-scan-user-dir":"false","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","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","fs.automatic.close":"true","yarn.nodemanager.hostname":"0.0.0.0","yarn.nodemanager.resource.memory.cgroups.swappiness":"0","ftp.stream-buffer-size":"4096","yarn.fail-fast":"false","yarn.timeline-service.app-aggregation-interval-secs":"15","hadoop.security.group.mapping.ldap.search.filter.user":"(&(objectClass=user)(sAMAccountName={0}))","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","yarn.resourcemanager.delegation.token.max-lifetime":"*********(redacted)","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.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.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.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","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","hadoop.security.dns.log-slow-lookups.enabled":"false","mapreduce.job.committer.setup.cleanup.needed":"true","mapreduce.job.running.reduce.limit":"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","fs.s3a.max.total.tasks":"5","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.nodemanager.health-checker.script.timeout-ms":"1200000","yarn.timeline-service.leveldb-timeline-store.start-time-write-cache-size":"10000","yarn.resourcemanager.resource-profiles.enabled":"false","yarn.timeline-service.hbase-schema.prefix":"prod.","fs.azure.authorization":"false","mapreduce.map.log.level":"INFO","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","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","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","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":"0,1","yarn.timeline-service.client.internal-timers-ttl-secs":"420","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","ipc.client.tcpnodelay":"true","ipc.client.rpc-timeout.ms":"0","yarn.nodemanager.webapp.rest-csrf.methods-to-ignore":"GET,OPTIONS,HEAD","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","fs.s3a.socket.recv.buffer":"8192","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.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","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","mapreduce.shuffle.ssl.enabled":"false","yarn.timeline-service.hbase.coprocessor.app-final-value-retention-milliseconds":"259200000","fs.s3a.committer.staging.abort.pending.uploads":"true","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","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":"100M","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","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","yarn.resourcemanager.ha.enabled":"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.federation.subcluster-resolver.class":"org.apache.hadoop.yarn.server.federation.resolver.DefaultSubClusterResolverImpl","yarn.resourcemanager.zk-state-store.parent-path":"/rmstore","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":"fail","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","fs.AbstractFileSystem.adl.impl":"org.apache.hadoop.fs.adl.Adl","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":"128","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.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.federation.registry.base-dir":"yarnfederation/","mapreduce.job.max.map":"-1","mapreduce.job.local-fs.single-disk-limit.bytes":"-1","mapreduce.job.ubertask.maxreduces":"1","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","yarn.client.nodemanager-connect.retry-interval-ms":"10000","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","yarn.timeline-service.webapp.rest-csrf.enabled":"false"},"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":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Private Build","java.vm.specification.version":"1.8","user.home":"/home/user1","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/amd64","user.dir":"/home/user1/workspace/spark-rapids-another","java.library.path":"/usr/local/cuda/lib64:/usr/java/packages/lib/amd64:/usr/lib/x86_64-linux-gnu/jni:/lib/x86_64-linux-gnu:/usr/lib/x86_64-linux-gnu:/usr/lib/jni:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.282-b08","jetty.git.hash":"e46af88704a893fc12cb0e3bf46e2c7b48a009e7","java.endorsed.dirs":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/endorsed","java.runtime.version":"1.8.0_282-8u282-b08-0ubuntu1~18.04-b08","java.vm.info":"mixed mode","java.ext.dirs":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"OpenJDK Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/resources.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/rt.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/sunrsasign.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jsse.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jce.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/charsets.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jfr.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"4.15.0-130-generic","sun.os.patch.level":"unknown","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","user.language":"en","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.print.PSPrinterJob","java.awt.graphicsenv":"sun.awt.X11GraphicsEnvironment","awt.toolkit":"sun.awt.X11.XToolkit","os.name":"Linux","java.vm.vendor":"Private Build","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"user1","java.vm.name":"OpenJDK 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master local[1] --conf spark.driver.memory=30g --class com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils --jars /home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar /home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar dsAndDf /home/user1/testeventlogDir 100001","java.home":"/usr/lib/jvm/java-8-openjdk-amd64/jre","java.version":"1.8.0_282","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-exec-2.3.7-core.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aopalliance-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/breeze-macros_2.12-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-yarn_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jaxb-api-2.2.11.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-scala_2.12-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okhttp-2.7.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-scheduling-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-jaxrs-json-provider-2.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-storageclass-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-sketch_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/flatbuffers-java-1.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-media-jaxb-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/chill_2.12-0.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-policy-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guava-14.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jline-2.14.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stax-api-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/chill-java-0.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/shims-0.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jdo-api-3.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-autoscaling-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/gson-2.2.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/py4j-0.10.9.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-core_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-memory-netty-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-jaxb-annotations-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-kubernetes_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-admin-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/slf4j-api-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stax2-api-3.1.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-core_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-xdr-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-common-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-apiextensions-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/antlr4-runtime-4.8-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-common-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/antlr-runtime-3.5.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-core-4.1.17.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mesos_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-streaming_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jsp-api-2.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-framework-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/activation-1.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/mesos-1.4.0-shaded-protobuf.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/pyrolite-4.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/libthrift-0.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-ipc-1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-scheduler-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.annotation-api-1.3.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-client-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-crypto-1.1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-util-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-events-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-databind-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/xbean-asm7-shaded-4.15.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-mapreduce-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/core-1.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okio-1.14.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-discovery-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zookeeper-3.4.14.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/woodstox-core-5.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.xml.bind-api-2.3.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-apps-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-service-rpc-3.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mllib-local_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-launcher_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/httpcore-4.4.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jcl-over-slf4j-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-tags_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-jdbc-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-client-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arpack_combined_all-0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/token-provider-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/joda-time-2.10.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-graphx_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-format-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/re2j-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kryo-shaded-4.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jpam-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-hadoop-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/super-csv-2.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/breeze_2.12-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-jackson_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-collections-3.2.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-core-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-beeline-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-ast_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-xml_2.12-1.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-server-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/bonecp-0.8.0.RELEASE.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/slf4j-log4j12-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-shims-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/leveldbjni-all-1.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-reflect-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-serde-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/audience-annotations-0.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-batch-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-coordination-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-common-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-paranamer-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/threeten-extra-1.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-client-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/compress-lzf-1.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javax.inject-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/conf/":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-core-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/accessors-smart-1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-math3-3.4.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/RoaringBitmap-0.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ehcache-3.3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-column-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.activation-api-1.2.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-configuration2-2.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-api-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-recipes-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-network-shuffle_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/xz-1.5.jar":"System Classpath","spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar":"Added By User","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/dropwizard-metrics-hadoop-metrics2-reporter-0.1.2.jar":"System Classpath","spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar":"Added By User","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aircompressor-0.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-graphite-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-jmx-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-server-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/protobuf-java-2.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-util_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-lang3-3.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-memory-core-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-vector-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/generex-1.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/httpclient-4.5.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-client-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-compress-1.20.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-metastore-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-auth-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-scalap_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.ws.rs-api-2.1.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ST4-4.0.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-locator-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-networking-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-extensions-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jaxb-runtime-2.3.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-compiler-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/shapeless_2.12-2.3.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-core-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-pool-1.5.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-metrics-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jta-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ivy-2.4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/paranamer-2.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/istack-commons-runtime-3.0.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-text-1.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-cli-1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-core-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-net-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-utils-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/transaction-api-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/automaton-1.11-8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-parser-combinators_2.12-1.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/cats-kernel_2.12-2.0.0-M4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/nimbus-jose-jwt-4.41.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-util-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-container-servlet-core-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-vector-code-gen-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.validation-api-2.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-jaxrs-base-2.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-common-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-rdbms-4.1.19.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/janino-3.0.16.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/algebra_2.12-2.0.0-M2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/velocity-1.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.servlet-api-4.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-jobclient-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/oro-2.0.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/htrace-core4-4.1.0-incubating.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-mapred-1.8.2-hadoop2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-datatype-jsr310-2.11.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-kvstore_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-repl_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jul-to-slf4j-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-codec-1.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-cli-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-hk2-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guice-4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javassist-3.25.0-GA.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-lang-2.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/macro-compat_2.12-1.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-io-2.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-hive-thriftserver_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-dbcp-1.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-core-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-0.23-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-encoding-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okhttp-3.12.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guice-servlet-4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-hdfs-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-daemon-1.0.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-library-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-api-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jsr305-3.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json-1.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-network-common_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zstd-jni-1.4.8-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/snappy-java-1.1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-macros_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-annotations-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-simplekdc-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-logging-1.1.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stream-2.9.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-httpclient-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-hive_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-jackson-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-format-2.4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-core-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json-smart-2.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javax.jdo-3.2.0-m3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zjsonpatch-0.3.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-pkix-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/opencsv-2.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/machinist_2.12-0.6.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aopalliance-repackaged-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-platform_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/netty-all-4.1.51.Final.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-admissionregistration-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-rbac-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/osgi-resource-locator-1.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-annotations-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-config-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/libfb303-0.9.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-unsafe_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-beanutils-1.9.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-tags_2.12-3.1.1-tests.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-llap-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mllib_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-identity-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/log4j-1.2.17.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-settings-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-collection-compat_2.12-2.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-jvm-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/derby-10.12.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-storage-api-2.7.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/JLargeArrays-1.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-api-jdo-4.2.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/JTransforms-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javolution-5.5.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/geronimo-jcache_1.0_spec-1.0-alpha-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/minlog-1.3.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-container-servlet-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/objenesis-2.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/logging-interceptor-3.12.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-compiler-3.0.16.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-certificates-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jodd-core-3.5.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-sql_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jcip-annotations-1.0-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-asn1-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-json-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.inject-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-server-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-dataformat-yaml-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-server-web-proxy-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-registry-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-catalyst_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/HikariCP-2.5.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-crypto-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/dnsjava-2.1.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/univocity-parsers-2.9.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/lz4-java-1.7.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/snakeyaml-1.24.jar":"System Classpath"}} -{"Event":"SparkListenerApplicationStart","App Name":"Rapids Spark Profiling Tool Unit Tests","App ID":"local-1622043423018","Timestamp":1622043422343,"User":"user1"} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/lib/jvm/java-8-openjdk-amd64/jre","Java Version":"1.8.0_282 (Private Build)","Scala Version":"version 2.12.10"},"Spark Properties":{"spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.scheduler.mode":"FIFO","spark.driver.memory":"30g","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423015"},"Hadoop Properties":{"hadoop.service.shutdown.timeout":"30s","yarn.resourcemanager.amlauncher.thread-count":"50","yarn.sharedcache.enabled":"false","fs.s3a.connection.maximum":"15","yarn.nodemanager.numa-awareness.numactl.cmd":"/usr/bin/numactl","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","mapreduce.framework.name":"local","yarn.sharedcache.uploader.server.thread-count":"50","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","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","hadoop.registry.secure":"false","hadoop.shell.safely.delete.limit.num.files":"100","dfs.bytes-per-checksum":"512","mapreduce.job.acl-view-job":" ","fs.s3a.s3guard.ddb.background.sleep":"25ms","fs.s3a.retry.limit":"${fs.s3a.attempts.maximum}","mapreduce.jobhistory.loadedjobs.cache.size":"5","fs.s3a.s3guard.ddb.table.create":"false","yarn.nodemanager.amrmproxy.enabled":"false","yarn.timeline-service.entity-group-fs-store.with-user-dir":"false","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.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.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","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","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.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.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","yarn.timeline-service.client.fd-clean-interval-secs":"60","yarn.resourcemanager.nodemanagers.heartbeat-interval-ms":"1000","hadoop.common.configuration.version":"3.0.0","fs.s3a.s3guard.ddb.table.capacity.read":"500","yarn.nodemanager.remote-app-log-dir-suffix":"logs","yarn.nodemanager.windows-container.cpu-limit.enabled":"false","yarn.nodemanager.runtime.linux.docker.privileged-containers.allowed":"false","file.blocksize":"67108864","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","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","fs.har.impl.disable.cache":"true","yarn.webapp.ui2.enable":"false","io.compression.codec.bzip2.library":"system-native","yarn.nodemanager.distributed-scheduling.enabled":"false","mapreduce.shuffle.connection-keep-alive.timeout":"5","yarn.resourcemanager.webapp.https.address":"${yarn.resourcemanager.hostname}:8090","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.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","fs.ftp.transfer.mode":"BLOCK_TRANSFER_MODE","ipc.server.log.slow.rpc":"false","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","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","hadoop.security.credential.clear-text-fallback":"true","yarn.nodemanager.collector-service.thread-count":"5","fs.azure.secure.mode":"false","mapreduce.jobhistory.joblist.cache.size":"20000","fs.ftp.host":"0.0.0.0","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":"false","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","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","yarn.client.load.resource-types.from-server":"false","ha.zookeeper.session-timeout.ms":"10000","tfile.io.chunk.size":"1048576","fs.s3a.s3guard.ddb.table.capacity.write":"100","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","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","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","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","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","mapreduce.job.max.split.locations":"15","yarn.resourcemanager.nm-container-queuing.max-queue-length":"15","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.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","fs.s3a.buffer.dir":"${hadoop.tmp.dir}/s3a","hadoop.ssl.enabled.protocols":"TLSv1,SSLv2Hello,TLSv1.1,TLSv1.2","mapreduce.jobhistory.admin.address":"0.0.0.0:10033","yarn.log-aggregation-status.time-out.ms":"600000","fs.s3a.assumed.role.sts.endpoint.region":"us-west-1","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.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","fs.azure.local.sas.key.mode":"false","ipc.maximum.data.length":"67108864","mapreduce.shuffle.max.threads":"0","yarn.router.pipeline.cache-max-size":"25","yarn.resourcemanager.nm-container-queuing.load-comparator":"QUEUE_LENGTH","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","mapreduce.jobhistory.http.policy":"HTTP_ONLY","yarn.sharedcache.store.in-memory.check-period-mins":"720","hadoop.security.group.mapping.ldap.ssl":"false","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.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.nodemanager.resourcemanager.minimum.version":"NONE","mapreduce.job.speculative.speculative-cap-running-tasks":"0.1","yarn.admin.acl":"*","yarn.nodemanager.recovery.supervised":"false","yarn.sharedcache.admin.thread-count":"1","yarn.resourcemanager.ha.automatic-failover.enabled":"true","mapreduce.reduce.skip.maxgroups":"0","mapreduce.reduce.shuffle.connect.timeout":"180000","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","hadoop.ssl.enabled":"false","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":"10","mapreduce.reduce.shuffle.input.buffer.percent":"0.70","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.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":"2147483647","yarn.resourcemanager.webapp.xfs-filter.xframe-options":"SAMEORIGIN","mapreduce.jobhistory.done-dir":"${yarn.app.mapreduce.am.staging-dir}/history/done","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","yarn.webapp.filter-entity-list-by-user":"false","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}","hadoop.workaround.non.threadsafe.getpwuid":"true","fs.df.interval":"60000","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","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","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.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","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.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","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","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.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","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","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":"${fs.s3a.attempts.maximum}","hadoop.http.authentication.type":"simple","mapreduce.job.cache.limit.max-resources":"0","mapreduce.task.userlog.limit.kb":"0","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","hadoop.user.group.static.mapping.overrides":"dr.who=;","fs.azure.sas.expiry.period":"90d","mapreduce.jobhistory.recovery.store.class":"org.apache.hadoop.mapreduce.v2.hs.HistoryServerFileSystemStateStoreService","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","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","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":"1000ms","seq.io.sort.factor":"100","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/","mapreduce.task.profile":"false","yarn.resourcemanager.fs.state-store.uri":"${hadoop.tmp.dir}/yarn/system/rmstore","mapreduce.jobhistory.always-scan-user-dir":"false","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","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","fs.automatic.close":"true","yarn.nodemanager.hostname":"0.0.0.0","yarn.nodemanager.resource.memory.cgroups.swappiness":"0","ftp.stream-buffer-size":"4096","yarn.fail-fast":"false","yarn.timeline-service.app-aggregation-interval-secs":"15","hadoop.security.group.mapping.ldap.search.filter.user":"(&(objectClass=user)(sAMAccountName={0}))","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","yarn.resourcemanager.delegation.token.max-lifetime":"*********(redacted)","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.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.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.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","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","hadoop.security.dns.log-slow-lookups.enabled":"false","mapreduce.job.committer.setup.cleanup.needed":"true","mapreduce.job.running.reduce.limit":"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","fs.s3a.max.total.tasks":"5","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.nodemanager.health-checker.script.timeout-ms":"1200000","yarn.timeline-service.leveldb-timeline-store.start-time-write-cache-size":"10000","yarn.resourcemanager.resource-profiles.enabled":"false","yarn.timeline-service.hbase-schema.prefix":"prod.","fs.azure.authorization":"false","mapreduce.map.log.level":"INFO","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","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","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","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":"0,1","yarn.timeline-service.client.internal-timers-ttl-secs":"420","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","ipc.client.tcpnodelay":"true","ipc.client.rpc-timeout.ms":"0","yarn.nodemanager.webapp.rest-csrf.methods-to-ignore":"GET,OPTIONS,HEAD","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","fs.s3a.socket.recv.buffer":"8192","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.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","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","mapreduce.shuffle.ssl.enabled":"false","yarn.timeline-service.hbase.coprocessor.app-final-value-retention-milliseconds":"259200000","fs.s3a.committer.staging.abort.pending.uploads":"true","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","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":"100M","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","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","yarn.resourcemanager.ha.enabled":"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.federation.subcluster-resolver.class":"org.apache.hadoop.yarn.server.federation.resolver.DefaultSubClusterResolverImpl","yarn.resourcemanager.zk-state-store.parent-path":"/rmstore","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":"fail","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","fs.AbstractFileSystem.adl.impl":"org.apache.hadoop.fs.adl.Adl","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":"128","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.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.federation.registry.base-dir":"yarnfederation/","mapreduce.job.max.map":"-1","mapreduce.job.local-fs.single-disk-limit.bytes":"-1","mapreduce.job.ubertask.maxreduces":"1","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","yarn.client.nodemanager-connect.retry-interval-ms":"10000","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","yarn.timeline-service.webapp.rest-csrf.enabled":"false"},"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":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Private Build","java.vm.specification.version":"1.8","user.home":"/home/user1","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/amd64","user.dir":"/home/user1/workspace/spark-rapids-another","java.library.path":"/usr/local/cuda/lib64:/usr/java/packages/lib/amd64:/usr/lib/x86_64-linux-gnu/jni:/lib/x86_64-linux-gnu:/usr/lib/x86_64-linux-gnu:/usr/lib/jni:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.282-b08","jetty.git.hash":"e46af88704a893fc12cb0e3bf46e2c7b48a009e7","java.endorsed.dirs":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/endorsed","java.runtime.version":"1.8.0_282-8u282-b08-0ubuntu1~18.04-b08","java.vm.info":"mixed mode","java.ext.dirs":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"OpenJDK Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/resources.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/rt.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/sunrsasign.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jsse.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jce.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/charsets.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jfr.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"4.15.0-130-generic","sun.os.patch.level":"unknown","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","user.language":"en","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.print.PSPrinterJob","java.awt.graphicsenv":"sun.awt.X11GraphicsEnvironment","awt.toolkit":"sun.awt.X11.XToolkit","os.name":"Linux","java.vm.vendor":"Private Build","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"user1","java.vm.name":"OpenJDK 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master local[1] --conf spark.driver.memory=30g --class com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils --jars /home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar /home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar dsAndDf /home/user1/testeventlogDir 100001","java.home":"/usr/lib/jvm/java-8-openjdk-amd64/jre","java.version":"1.8.0_282","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-exec-2.3.7-core.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aopalliance-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/breeze-macros_2.12-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-yarn_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jaxb-api-2.2.11.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-scala_2.12-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okhttp-2.7.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-scheduling-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-jaxrs-json-provider-2.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-storageclass-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-sketch_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/flatbuffers-java-1.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-media-jaxb-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/chill_2.12-0.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-policy-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guava-14.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jline-2.14.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stax-api-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/chill-java-0.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/shims-0.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jdo-api-3.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-autoscaling-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/gson-2.2.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/py4j-0.10.9.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-core_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-memory-netty-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-jaxb-annotations-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-kubernetes_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-admin-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/slf4j-api-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stax2-api-3.1.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-core_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-xdr-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-common-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-apiextensions-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/antlr4-runtime-4.8-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-common-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/antlr-runtime-3.5.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-core-4.1.17.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mesos_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-streaming_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jsp-api-2.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-framework-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/activation-1.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/mesos-1.4.0-shaded-protobuf.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/pyrolite-4.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/libthrift-0.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-ipc-1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-scheduler-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.annotation-api-1.3.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-client-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-crypto-1.1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-util-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-events-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-databind-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/xbean-asm7-shaded-4.15.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-mapreduce-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/core-1.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okio-1.14.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-discovery-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zookeeper-3.4.14.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/woodstox-core-5.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.xml.bind-api-2.3.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-apps-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-service-rpc-3.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mllib-local_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-launcher_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/httpcore-4.4.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jcl-over-slf4j-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-tags_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-jdbc-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-client-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arpack_combined_all-0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/token-provider-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/joda-time-2.10.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-graphx_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-format-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/re2j-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kryo-shaded-4.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jpam-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-hadoop-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/super-csv-2.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/breeze_2.12-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-jackson_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-collections-3.2.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-core-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-beeline-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-ast_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-xml_2.12-1.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-server-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/bonecp-0.8.0.RELEASE.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/slf4j-log4j12-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-shims-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/leveldbjni-all-1.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-reflect-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-serde-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/audience-annotations-0.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-batch-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-coordination-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-common-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-paranamer-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/threeten-extra-1.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-client-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/compress-lzf-1.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javax.inject-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/conf/":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-core-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/accessors-smart-1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-math3-3.4.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/RoaringBitmap-0.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ehcache-3.3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-column-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.activation-api-1.2.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-configuration2-2.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-api-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-recipes-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-network-shuffle_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/xz-1.5.jar":"System Classpath","spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar":"Added By User","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/dropwizard-metrics-hadoop-metrics2-reporter-0.1.2.jar":"System Classpath","spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar":"Added By User","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aircompressor-0.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-graphite-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-jmx-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-server-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/protobuf-java-2.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-util_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-lang3-3.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-memory-core-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-vector-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/generex-1.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/httpclient-4.5.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-client-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-compress-1.20.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-metastore-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-auth-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-scalap_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.ws.rs-api-2.1.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ST4-4.0.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-locator-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-networking-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-extensions-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jaxb-runtime-2.3.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-compiler-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/shapeless_2.12-2.3.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-core-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-pool-1.5.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-metrics-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jta-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ivy-2.4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/paranamer-2.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/istack-commons-runtime-3.0.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-text-1.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-cli-1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-core-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-net-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-utils-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/transaction-api-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/automaton-1.11-8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-parser-combinators_2.12-1.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/cats-kernel_2.12-2.0.0-M4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/nimbus-jose-jwt-4.41.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-util-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-container-servlet-core-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-vector-code-gen-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.validation-api-2.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-jaxrs-base-2.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-common-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-rdbms-4.1.19.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/janino-3.0.16.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/algebra_2.12-2.0.0-M2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/velocity-1.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.servlet-api-4.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-jobclient-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/oro-2.0.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/htrace-core4-4.1.0-incubating.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-mapred-1.8.2-hadoop2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-datatype-jsr310-2.11.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-kvstore_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-repl_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jul-to-slf4j-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-codec-1.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-cli-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-hk2-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guice-4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javassist-3.25.0-GA.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-lang-2.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/macro-compat_2.12-1.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-io-2.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-hive-thriftserver_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-dbcp-1.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-core-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-0.23-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-encoding-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okhttp-3.12.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guice-servlet-4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-hdfs-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-daemon-1.0.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-library-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-api-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jsr305-3.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json-1.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-network-common_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zstd-jni-1.4.8-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/snappy-java-1.1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-macros_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-annotations-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-simplekdc-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-logging-1.1.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stream-2.9.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-httpclient-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-hive_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-jackson-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-format-2.4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-core-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json-smart-2.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javax.jdo-3.2.0-m3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zjsonpatch-0.3.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-pkix-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/opencsv-2.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/machinist_2.12-0.6.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aopalliance-repackaged-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-platform_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/netty-all-4.1.51.Final.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-admissionregistration-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-rbac-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/osgi-resource-locator-1.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-annotations-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-config-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/libfb303-0.9.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-unsafe_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-beanutils-1.9.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-tags_2.12-3.1.1-tests.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-llap-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mllib_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-identity-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/log4j-1.2.17.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-settings-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-collection-compat_2.12-2.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-jvm-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/derby-10.12.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-storage-api-2.7.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/JLargeArrays-1.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-api-jdo-4.2.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/JTransforms-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javolution-5.5.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/geronimo-jcache_1.0_spec-1.0-alpha-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/minlog-1.3.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-container-servlet-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/objenesis-2.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/logging-interceptor-3.12.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-compiler-3.0.16.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-certificates-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jodd-core-3.5.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-sql_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jcip-annotations-1.0-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-asn1-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-json-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.inject-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-server-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-dataformat-yaml-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-server-web-proxy-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-registry-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-catalyst_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/HikariCP-2.5.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-crypto-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/dnsjava-2.1.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/univocity-parsers-2.9.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/lz4-java-1.7.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/snakeyaml-1.24.jar":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Rapids Spark Profiling Tool Unit Tests","App ID":"local-1622043423015","Timestamp":1622043422343,"User":"user1"} {"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":0,"description":"json at QualificationInfoUtils.scala:76","details":"org.apache.spark.sql.DataFrameWriter.json(DataFrameWriter.scala:853)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1(QualificationInfoUtils.scala:76)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1$adapted(QualificationInfoUtils.scala:73)\norg.apache.spark.rapids.TestUtils$.withTempPath(TestUtils.scala:31)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genDatasetEventLog(QualificationInfoUtils.scala:73)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:185)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nExecute InsertIntoHadoopFsRelationCommand (8)\n+- * SerializeFromObject (7)\n +- * MapElements (6)\n +- * Filter (5)\n +- * DeserializeToObject (4)\n +- * Project (3)\n +- * SerializeFromObject (2)\n +- Scan (1)\n\n\n(1) Scan\nOutput [1]: [obj#3]\nArguments: obj#3: scala.Tuple3, ParallelCollectionRDD[0] at parallelize at QualificationInfoUtils.scala:64\n\n(2) SerializeFromObject [codegen id : 1]\nInput [1]: [obj#3]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._1, true, false) AS _1#4, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._2, true, false) AS _2#5, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._3 AS _3#6]\n\n(3) Project [codegen id : 1]\nOutput [3]: [_1#4 AS name#11, _2#5 AS friend#12, _3#6 AS age#13]\nInput [3]: [_1#4, _2#5, _3#6]\n\n(4) DeserializeToObject [codegen id : 1]\nInput [3]: [name#11, friend#12, age#13]\nArguments: newInstance(class com.nvidia.spark.rapids.tool.profiling.RapidsFriends), obj#30: com.nvidia.spark.rapids.tool.profiling.RapidsFriends\n\n(5) Filter [codegen id : 1]\nInput [1]: [obj#30]\nCondition : com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$$$Lambda$1503/443638755@31f31b20.apply\n\n(6) MapElements [codegen id : 1]\nInput [1]: [obj#30]\nArguments: com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$$$Lambda$1505/1108374296@6723cce7, obj#31: scala.Tuple2\n\n(7) SerializeFromObject [codegen id : 1]\nInput [1]: [obj#31]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1, true, false) AS _1#32, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._2 AS _2#33]\n\n(8) Execute InsertIntoHadoopFsRelationCommand\nInput [2]: [_1#32, _2#33]\nArguments: file:/tmp/spark-4a9e661e-fc37-4f5f-8bbc-1647dbaebbaf, false, JSON, Map(path -> /tmp/spark-4a9e661e-fc37-4f5f-8bbc-1647dbaebbaf), ErrorIfExists, [_1, _2]\n\n","sparkPlanInfo":{"nodeName":"Execute InsertIntoHadoopFsRelationCommand","simpleString":"Execute InsertIntoHadoopFsRelationCommand file:/tmp/spark-4a9e661e-fc37-4f5f-8bbc-1647dbaebbaf, false, JSON, Map(path -> /tmp/spark-4a9e661e-fc37-4f5f-8bbc-1647dbaebbaf), ErrorIfExists, [_1, _2]","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1, true, false) AS _1#32, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._2 AS _2#33]","children":[{"nodeName":"MapElements","simpleString":"MapElements com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$$$Lambda$1505/1108374296@6723cce7, obj#31: scala.Tuple2","children":[{"nodeName":"Filter","simpleString":"Filter com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$$$Lambda$1503/443638755@31f31b20.apply","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject newInstance(class com.nvidia.spark.rapids.tool.profiling.RapidsFriends), obj#30: com.nvidia.spark.rapids.tool.profiling.RapidsFriends","children":[{"nodeName":"Project","simpleString":"Project [_1#4 AS name#11, _2#5 AS friend#12, _3#6 AS age#13]","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._1, true, false) AS _1#4, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._2, true, false) AS _2#5, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._3 AS _3#6]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Scan","simpleString":"Scan[obj#3]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":6,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":5,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":4,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of written files","accumulatorId":0,"metricType":"sum"},{"name":"written output","accumulatorId":1,"metricType":"size"},{"name":"number of output rows","accumulatorId":2,"metricType":"sum"},{"name":"number of dynamic part","accumulatorId":3,"metricType":"sum"}]},"time":1622043425909} -{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1622043426367,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"json at QualificationInfoUtils.scala:76","Number of Tasks":12,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"Scan\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at QualificationInfoUtils.scala:64","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameWriter.json(DataFrameWriter.scala:853)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1(QualificationInfoUtils.scala:76)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1$adapted(QualificationInfoUtils.scala:73)\norg.apache.spark.rapids.TestUtils$.withTempPath(TestUtils.scala:31)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genDatasetEventLog(QualificationInfoUtils.scala:73)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:185)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[0],"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"1\",\"name\":\"Execute InsertIntoHadoopFsRelationCommand\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"0","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"json at QualificationInfoUtils.scala:76","Number of Tasks":12,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"Scan\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at QualificationInfoUtils.scala:64","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameWriter.json(DataFrameWriter.scala:853)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1(QualificationInfoUtils.scala:76)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1$adapted(QualificationInfoUtils.scala:73)\norg.apache.spark.rapids.TestUtils$.withTempPath(TestUtils.scala:31)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genDatasetEventLog(QualificationInfoUtils.scala:73)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:185)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043426387,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"1\",\"name\":\"Execute InsertIntoHadoopFsRelationCommand\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"0","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"}} +{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1622043426367,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"json at QualificationInfoUtils.scala:76","Number of Tasks":12,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"Scan\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at QualificationInfoUtils.scala:64","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameWriter.json(DataFrameWriter.scala:853)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1(QualificationInfoUtils.scala:76)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1$adapted(QualificationInfoUtils.scala:73)\norg.apache.spark.rapids.TestUtils$.withTempPath(TestUtils.scala:31)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genDatasetEventLog(QualificationInfoUtils.scala:73)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:185)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[0],"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"1\",\"name\":\"Execute InsertIntoHadoopFsRelationCommand\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"0","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423015"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"json at QualificationInfoUtils.scala:76","Number of Tasks":12,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"Scan\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at QualificationInfoUtils.scala:64","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameWriter.json(DataFrameWriter.scala:853)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1(QualificationInfoUtils.scala:76)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1$adapted(QualificationInfoUtils.scala:73)\norg.apache.spark.rapids.TestUtils$.withTempPath(TestUtils.scala:31)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genDatasetEventLog(QualificationInfoUtils.scala:73)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:185)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043426387,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"1\",\"name\":\"Execute InsertIntoHadoopFsRelationCommand\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"0","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423015"}} {"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1622043426546,"Executor ID":"driver","Host":"10.10.10.100","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} {"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1622043426567,"Executor ID":"driver","Host":"10.10.10.100","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} {"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1622043426568,"Executor ID":"driver","Host":"10.10.10.100","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} @@ -39,9 +39,9 @@ {"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":0,"accumUpdates":[[0,12],[1,16560],[2,720],[3,0]]} {"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":0,"time":1622043427215} {"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":1,"description":"count at QualificationInfoUtils.scala:94","details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\n* HashAggregate (15)\n+- Exchange (14)\n +- * HashAggregate (13)\n +- * Project (12)\n +- * SortMergeJoin Inner (11)\n :- * Sort (5)\n : +- Exchange (4)\n : +- * Project (3)\n : +- * SerializeFromObject (2)\n : +- Scan (1)\n +- * Sort (10)\n +- Exchange (9)\n +- * Project (8)\n +- * SerializeFromObject (7)\n +- Scan (6)\n\n\n(1) Scan\nOutput [1]: [obj#40]\nArguments: obj#40: int, ParallelCollectionRDD[5] at makeRDD at QualificationInfoUtils.scala:92\n\n(2) SerializeFromObject [codegen id : 1]\nInput [1]: [obj#40]\nArguments: [input[0, int, false] AS value#41]\n\n(3) Project [codegen id : 1]\nOutput [1]: [value#41 AS a#51]\nInput [1]: [value#41]\n\n(4) Exchange\nInput [1]: [a#51]\nArguments: hashpartitioning(a#51, 200), ENSURE_REQUIREMENTS, [id=#93]\n\n(5) Sort [codegen id : 2]\nInput [1]: [a#51]\nArguments: [a#51 ASC NULLS FIRST], false, 0\n\n(6) Scan\nOutput [1]: [obj#46]\nArguments: obj#46: int, ParallelCollectionRDD[6] at makeRDD at QualificationInfoUtils.scala:93\n\n(7) SerializeFromObject [codegen id : 3]\nInput [1]: [obj#46]\nArguments: [input[0, int, false] AS value#47]\n\n(8) Project [codegen id : 3]\nOutput [1]: [value#47 AS b#53]\nInput [1]: [value#47]\n\n(9) Exchange\nInput [1]: [b#53]\nArguments: hashpartitioning(b#53, 200), ENSURE_REQUIREMENTS, [id=#102]\n\n(10) Sort [codegen id : 4]\nInput [1]: [b#53]\nArguments: [b#53 ASC NULLS FIRST], false, 0\n\n(11) SortMergeJoin [codegen id : 5]\nLeft keys [1]: [a#51]\nRight keys [1]: [b#53]\nJoin condition: None\n\n(12) Project [codegen id : 5]\nOutput: []\nInput [2]: [a#51, b#53]\n\n(13) HashAggregate [codegen id : 5]\nInput: []\nKeys: []\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count#64L]\nResults [1]: [count#65L]\n\n(14) Exchange\nInput [1]: [count#65L]\nArguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#111]\n\n(15) HashAggregate [codegen id : 6]\nInput [1]: [count#65L]\nKeys: []\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#61L]\nResults [1]: [count(1)#61L AS count#62L]\n\n","sparkPlanInfo":{"nodeName":"WholeStageCodegen (6)","simpleString":"WholeStageCodegen (6)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Exchange","simpleString":"Exchange SinglePartition, ENSURE_REQUIREMENTS, [id=#111]","children":[{"nodeName":"WholeStageCodegen (5)","simpleString":"WholeStageCodegen (5)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[], functions=[partial_count(1)])","children":[{"nodeName":"Project","simpleString":"Project","children":[{"nodeName":"SortMergeJoin","simpleString":"SortMergeJoin [a#51], [b#53], Inner","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"Sort","simpleString":"Sort [a#51 ASC NULLS FIRST], false, 0","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(a#51, 200), ENSURE_REQUIREMENTS, [id=#93]","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [value#41 AS a#51]","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [input[0, int, false] AS value#41]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Scan","simpleString":"Scan[obj#40]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":116,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":115,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":74,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":75,"metricType":"nsTiming"},{"name":"records read","accumulatorId":72,"metricType":"sum"},{"name":"local bytes read","accumulatorId":70,"metricType":"size"},{"name":"fetch wait time","accumulatorId":71,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":68,"metricType":"size"},{"name":"local blocks read","accumulatorId":67,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":66,"metricType":"sum"},{"name":"data size","accumulatorId":65,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":69,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":73,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"sort time","accumulatorId":112,"metricType":"timing"},{"name":"peak memory","accumulatorId":113,"metricType":"size"},{"name":"spill size","accumulatorId":114,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":111,"metricType":"timing"}]}],"metadata":{},"metrics":[]},{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"Sort","simpleString":"Sort [b#53 ASC NULLS FIRST], false, 0","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(b#53, 200), ENSURE_REQUIREMENTS, [id=#102]","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"Project","simpleString":"Project [value#47 AS b#53]","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [input[0, int, false] AS value#47]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Scan","simpleString":"Scan[obj#46]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":122,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":121,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":85,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":86,"metricType":"nsTiming"},{"name":"records read","accumulatorId":83,"metricType":"sum"},{"name":"local bytes read","accumulatorId":81,"metricType":"size"},{"name":"fetch wait time","accumulatorId":82,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":79,"metricType":"size"},{"name":"local blocks read","accumulatorId":78,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":77,"metricType":"sum"},{"name":"data size","accumulatorId":76,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":80,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":84,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"sort time","accumulatorId":118,"metricType":"timing"},{"name":"peak memory","accumulatorId":119,"metricType":"size"},{"name":"spill size","accumulatorId":120,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":117,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":110,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":107,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":108,"metricType":"timing"},{"name":"peak memory","accumulatorId":106,"metricType":"size"},{"name":"number of output rows","accumulatorId":105,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":109,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":104,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":96,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":97,"metricType":"nsTiming"},{"name":"records read","accumulatorId":94,"metricType":"sum"},{"name":"local bytes read","accumulatorId":92,"metricType":"size"},{"name":"fetch wait time","accumulatorId":93,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":90,"metricType":"size"},{"name":"local blocks read","accumulatorId":89,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":88,"metricType":"sum"},{"name":"data size","accumulatorId":87,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":91,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":95,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":101,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":102,"metricType":"timing"},{"name":"peak memory","accumulatorId":100,"metricType":"size"},{"name":"number of output rows","accumulatorId":99,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":103,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":98,"metricType":"timing"}]},"time":1622043427414} -{"Event":"SparkListenerJobStart","Job ID":1,"Submission Time":1622043427637,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"43\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"15\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:93","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"39\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"14\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:92","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"29\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":200,"RDD Info":[{"RDD ID":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[18],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[14],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"20\",\"name\":\"WholeStageCodegen (5)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"35\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"25\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"20\",\"name\":\"WholeStageCodegen (5)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[11,16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[1,2],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":1,"RDD Info":[{"RDD ID":22,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"44\",\"name\":\"mapPartitionsInternal\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":21,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"16\",\"name\":\"WholeStageCodegen (6)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"19\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[3],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[1,2,3,4],"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"43\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"15\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:93","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"39\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043427644,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"14\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:92","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"29\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043427656,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"}} +{"Event":"SparkListenerJobStart","Job ID":1,"Submission Time":1622043427637,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"43\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"15\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:93","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"39\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"14\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:92","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"29\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":200,"RDD Info":[{"RDD ID":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[18],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[14],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"20\",\"name\":\"WholeStageCodegen (5)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"35\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"25\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"20\",\"name\":\"WholeStageCodegen (5)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[11,16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[1,2],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":1,"RDD Info":[{"RDD ID":22,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"44\",\"name\":\"mapPartitionsInternal\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":21,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"16\",\"name\":\"WholeStageCodegen (6)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"19\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[3],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[1,2,3,4],"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423015"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"43\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"15\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:93","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"39\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043427644,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423015"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"14\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:92","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"29\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043427656,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423015"}} {"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":0,"Attempt":0,"Launch Time":1622043427655,"Executor ID":"driver","Host":"10.10.10.100","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} {"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":1,"Attempt":0,"Launch Time":1622043427661,"Executor ID":"driver","Host":"10.10.10.100","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} {"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":2,"Attempt":0,"Launch Time":1622043427661,"Executor ID":"driver","Host":"10.10.10.100","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} diff --git a/core/src/test/resources/spark-events-qualification/cluster_information/eventlog_3nodes_12cores_variable_cores b/core/src/test/resources/spark-events-qualification/cluster_information/eventlog_3nodes_12cores_variable_cores index 60380f829..5fde48d2a 100644 --- a/core/src/test/resources/spark-events-qualification/cluster_information/eventlog_3nodes_12cores_variable_cores +++ b/core/src/test/resources/spark-events-qualification/cluster_information/eventlog_3nodes_12cores_variable_cores @@ -7,11 +7,11 @@ {"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"0","Host":"10.59.184.244","Port":43349},"Maximum Memory":11884245811,"Timestamp":1702417718124,"Maximum Onheap Memory":11884245811,"Maximum Offheap Memory":0} {"Event":"SparkListenerExecutorAdded","Timestamp":1702417716738,"Executor ID":"2","Executor Info":{"Host":"10.59.184.267","Total Cores":8,"Log Urls":{"stdout":"http://10.59.184.244:40000/logPage/?appId=app-20231212214826-0000&executorId=0&logType=stdout","stderr":"http://10.59.184.244:40000/logPage/?appId=app-20231212214826-0000&executorId=0&logType=stderr"},"Attributes":{},"Resources":{},"Resource Profile Id":0}} {"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"2","Host":"10.59.184.267","Port":43349},"Maximum Memory":11884245811,"Timestamp":1702417718124,"Maximum Onheap Memory":11884245811,"Maximum Offheap Memory":0} -{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/lib/jvm/java-8-openjdk-amd64/jre","Java Version":"1.8.0_282 (Private Build)","Scala Version":"version 2.12.10"},"Spark Properties":{"spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.scheduler.mode":"FIFO","spark.driver.memory":"30g","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"},"Hadoop Properties":{"hadoop.service.shutdown.timeout":"30s","yarn.resourcemanager.amlauncher.thread-count":"50","yarn.sharedcache.enabled":"false","fs.s3a.connection.maximum":"15","yarn.nodemanager.numa-awareness.numactl.cmd":"/usr/bin/numactl","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","mapreduce.framework.name":"local","yarn.sharedcache.uploader.server.thread-count":"50","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","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","hadoop.registry.secure":"false","hadoop.shell.safely.delete.limit.num.files":"100","dfs.bytes-per-checksum":"512","mapreduce.job.acl-view-job":" ","fs.s3a.s3guard.ddb.background.sleep":"25ms","fs.s3a.retry.limit":"${fs.s3a.attempts.maximum}","mapreduce.jobhistory.loadedjobs.cache.size":"5","fs.s3a.s3guard.ddb.table.create":"false","yarn.nodemanager.amrmproxy.enabled":"false","yarn.timeline-service.entity-group-fs-store.with-user-dir":"false","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.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.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","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","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.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.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","yarn.timeline-service.client.fd-clean-interval-secs":"60","yarn.resourcemanager.nodemanagers.heartbeat-interval-ms":"1000","hadoop.common.configuration.version":"3.0.0","fs.s3a.s3guard.ddb.table.capacity.read":"500","yarn.nodemanager.remote-app-log-dir-suffix":"logs","yarn.nodemanager.windows-container.cpu-limit.enabled":"false","yarn.nodemanager.runtime.linux.docker.privileged-containers.allowed":"false","file.blocksize":"67108864","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","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","fs.har.impl.disable.cache":"true","yarn.webapp.ui2.enable":"false","io.compression.codec.bzip2.library":"system-native","yarn.nodemanager.distributed-scheduling.enabled":"false","mapreduce.shuffle.connection-keep-alive.timeout":"5","yarn.resourcemanager.webapp.https.address":"${yarn.resourcemanager.hostname}:8090","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.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","fs.ftp.transfer.mode":"BLOCK_TRANSFER_MODE","ipc.server.log.slow.rpc":"false","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","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","hadoop.security.credential.clear-text-fallback":"true","yarn.nodemanager.collector-service.thread-count":"5","fs.azure.secure.mode":"false","mapreduce.jobhistory.joblist.cache.size":"20000","fs.ftp.host":"0.0.0.0","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":"false","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","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","yarn.client.load.resource-types.from-server":"false","ha.zookeeper.session-timeout.ms":"10000","tfile.io.chunk.size":"1048576","fs.s3a.s3guard.ddb.table.capacity.write":"100","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","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","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","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","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","mapreduce.job.max.split.locations":"15","yarn.resourcemanager.nm-container-queuing.max-queue-length":"15","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.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","fs.s3a.buffer.dir":"${hadoop.tmp.dir}/s3a","hadoop.ssl.enabled.protocols":"TLSv1,SSLv2Hello,TLSv1.1,TLSv1.2","mapreduce.jobhistory.admin.address":"0.0.0.0:10033","yarn.log-aggregation-status.time-out.ms":"600000","fs.s3a.assumed.role.sts.endpoint.region":"us-west-1","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.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","fs.azure.local.sas.key.mode":"false","ipc.maximum.data.length":"67108864","mapreduce.shuffle.max.threads":"0","yarn.router.pipeline.cache-max-size":"25","yarn.resourcemanager.nm-container-queuing.load-comparator":"QUEUE_LENGTH","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","mapreduce.jobhistory.http.policy":"HTTP_ONLY","yarn.sharedcache.store.in-memory.check-period-mins":"720","hadoop.security.group.mapping.ldap.ssl":"false","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.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.nodemanager.resourcemanager.minimum.version":"NONE","mapreduce.job.speculative.speculative-cap-running-tasks":"0.1","yarn.admin.acl":"*","yarn.nodemanager.recovery.supervised":"false","yarn.sharedcache.admin.thread-count":"1","yarn.resourcemanager.ha.automatic-failover.enabled":"true","mapreduce.reduce.skip.maxgroups":"0","mapreduce.reduce.shuffle.connect.timeout":"180000","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","hadoop.ssl.enabled":"false","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":"10","mapreduce.reduce.shuffle.input.buffer.percent":"0.70","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.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":"2147483647","yarn.resourcemanager.webapp.xfs-filter.xframe-options":"SAMEORIGIN","mapreduce.jobhistory.done-dir":"${yarn.app.mapreduce.am.staging-dir}/history/done","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","yarn.webapp.filter-entity-list-by-user":"false","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}","hadoop.workaround.non.threadsafe.getpwuid":"true","fs.df.interval":"60000","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","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","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.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","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.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","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","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.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","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","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":"${fs.s3a.attempts.maximum}","hadoop.http.authentication.type":"simple","mapreduce.job.cache.limit.max-resources":"0","mapreduce.task.userlog.limit.kb":"0","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","hadoop.user.group.static.mapping.overrides":"dr.who=;","fs.azure.sas.expiry.period":"90d","mapreduce.jobhistory.recovery.store.class":"org.apache.hadoop.mapreduce.v2.hs.HistoryServerFileSystemStateStoreService","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","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","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":"1000ms","seq.io.sort.factor":"100","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/","mapreduce.task.profile":"false","yarn.resourcemanager.fs.state-store.uri":"${hadoop.tmp.dir}/yarn/system/rmstore","mapreduce.jobhistory.always-scan-user-dir":"false","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","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","fs.automatic.close":"true","yarn.nodemanager.hostname":"0.0.0.0","yarn.nodemanager.resource.memory.cgroups.swappiness":"0","ftp.stream-buffer-size":"4096","yarn.fail-fast":"false","yarn.timeline-service.app-aggregation-interval-secs":"15","hadoop.security.group.mapping.ldap.search.filter.user":"(&(objectClass=user)(sAMAccountName={0}))","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","yarn.resourcemanager.delegation.token.max-lifetime":"*********(redacted)","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.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.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.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","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","hadoop.security.dns.log-slow-lookups.enabled":"false","mapreduce.job.committer.setup.cleanup.needed":"true","mapreduce.job.running.reduce.limit":"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","fs.s3a.max.total.tasks":"5","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.nodemanager.health-checker.script.timeout-ms":"1200000","yarn.timeline-service.leveldb-timeline-store.start-time-write-cache-size":"10000","yarn.resourcemanager.resource-profiles.enabled":"false","yarn.timeline-service.hbase-schema.prefix":"prod.","fs.azure.authorization":"false","mapreduce.map.log.level":"INFO","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","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","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","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":"0,1","yarn.timeline-service.client.internal-timers-ttl-secs":"420","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","ipc.client.tcpnodelay":"true","ipc.client.rpc-timeout.ms":"0","yarn.nodemanager.webapp.rest-csrf.methods-to-ignore":"GET,OPTIONS,HEAD","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","fs.s3a.socket.recv.buffer":"8192","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.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","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","mapreduce.shuffle.ssl.enabled":"false","yarn.timeline-service.hbase.coprocessor.app-final-value-retention-milliseconds":"259200000","fs.s3a.committer.staging.abort.pending.uploads":"true","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","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":"100M","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","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","yarn.resourcemanager.ha.enabled":"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.federation.subcluster-resolver.class":"org.apache.hadoop.yarn.server.federation.resolver.DefaultSubClusterResolverImpl","yarn.resourcemanager.zk-state-store.parent-path":"/rmstore","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":"fail","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","fs.AbstractFileSystem.adl.impl":"org.apache.hadoop.fs.adl.Adl","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":"128","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.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.federation.registry.base-dir":"yarnfederation/","mapreduce.job.max.map":"-1","mapreduce.job.local-fs.single-disk-limit.bytes":"-1","mapreduce.job.ubertask.maxreduces":"1","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","yarn.client.nodemanager-connect.retry-interval-ms":"10000","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","yarn.timeline-service.webapp.rest-csrf.enabled":"false"},"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":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Private Build","java.vm.specification.version":"1.8","user.home":"/home/user1","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/amd64","user.dir":"/home/user1/workspace/spark-rapids-another","java.library.path":"/usr/local/cuda/lib64:/usr/java/packages/lib/amd64:/usr/lib/x86_64-linux-gnu/jni:/lib/x86_64-linux-gnu:/usr/lib/x86_64-linux-gnu:/usr/lib/jni:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.282-b08","jetty.git.hash":"e46af88704a893fc12cb0e3bf46e2c7b48a009e7","java.endorsed.dirs":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/endorsed","java.runtime.version":"1.8.0_282-8u282-b08-0ubuntu1~18.04-b08","java.vm.info":"mixed mode","java.ext.dirs":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"OpenJDK Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/resources.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/rt.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/sunrsasign.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jsse.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jce.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/charsets.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jfr.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"4.15.0-130-generic","sun.os.patch.level":"unknown","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","user.language":"en","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.print.PSPrinterJob","java.awt.graphicsenv":"sun.awt.X11GraphicsEnvironment","awt.toolkit":"sun.awt.X11.XToolkit","os.name":"Linux","java.vm.vendor":"Private Build","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"user1","java.vm.name":"OpenJDK 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master local[1] --conf spark.driver.memory=30g --class com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils --jars /home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar /home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar dsAndDf /home/user1/testeventlogDir 100001","java.home":"/usr/lib/jvm/java-8-openjdk-amd64/jre","java.version":"1.8.0_282","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-exec-2.3.7-core.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aopalliance-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/breeze-macros_2.12-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-yarn_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jaxb-api-2.2.11.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-scala_2.12-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okhttp-2.7.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-scheduling-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-jaxrs-json-provider-2.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-storageclass-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-sketch_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/flatbuffers-java-1.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-media-jaxb-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/chill_2.12-0.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-policy-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guava-14.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jline-2.14.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stax-api-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/chill-java-0.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/shims-0.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jdo-api-3.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-autoscaling-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/gson-2.2.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/py4j-0.10.9.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-core_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-memory-netty-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-jaxb-annotations-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-kubernetes_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-admin-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/slf4j-api-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stax2-api-3.1.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-core_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-xdr-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-common-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-apiextensions-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/antlr4-runtime-4.8-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-common-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/antlr-runtime-3.5.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-core-4.1.17.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mesos_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-streaming_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jsp-api-2.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-framework-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/activation-1.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/mesos-1.4.0-shaded-protobuf.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/pyrolite-4.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/libthrift-0.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-ipc-1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-scheduler-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.annotation-api-1.3.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-client-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-crypto-1.1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-util-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-events-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-databind-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/xbean-asm7-shaded-4.15.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-mapreduce-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/core-1.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okio-1.14.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-discovery-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zookeeper-3.4.14.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/woodstox-core-5.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.xml.bind-api-2.3.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-apps-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-service-rpc-3.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mllib-local_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-launcher_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/httpcore-4.4.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jcl-over-slf4j-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-tags_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-jdbc-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-client-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arpack_combined_all-0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/token-provider-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/joda-time-2.10.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-graphx_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-format-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/re2j-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kryo-shaded-4.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jpam-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-hadoop-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/super-csv-2.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/breeze_2.12-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-jackson_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-collections-3.2.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-core-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-beeline-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-ast_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-xml_2.12-1.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-server-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/bonecp-0.8.0.RELEASE.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/slf4j-log4j12-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-shims-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/leveldbjni-all-1.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-reflect-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-serde-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/audience-annotations-0.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-batch-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-coordination-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-common-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-paranamer-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/threeten-extra-1.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-client-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/compress-lzf-1.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javax.inject-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/conf/":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-core-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/accessors-smart-1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-math3-3.4.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/RoaringBitmap-0.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ehcache-3.3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-column-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.activation-api-1.2.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-configuration2-2.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-api-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-recipes-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-network-shuffle_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/xz-1.5.jar":"System Classpath","spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar":"Added By User","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/dropwizard-metrics-hadoop-metrics2-reporter-0.1.2.jar":"System Classpath","spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar":"Added By User","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aircompressor-0.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-graphite-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-jmx-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-server-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/protobuf-java-2.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-util_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-lang3-3.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-memory-core-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-vector-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/generex-1.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/httpclient-4.5.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-client-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-compress-1.20.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-metastore-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-auth-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-scalap_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.ws.rs-api-2.1.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ST4-4.0.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-locator-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-networking-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-extensions-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jaxb-runtime-2.3.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-compiler-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/shapeless_2.12-2.3.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-core-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-pool-1.5.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-metrics-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jta-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ivy-2.4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/paranamer-2.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/istack-commons-runtime-3.0.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-text-1.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-cli-1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-core-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-net-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-utils-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/transaction-api-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/automaton-1.11-8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-parser-combinators_2.12-1.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/cats-kernel_2.12-2.0.0-M4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/nimbus-jose-jwt-4.41.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-util-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-container-servlet-core-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-vector-code-gen-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.validation-api-2.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-jaxrs-base-2.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-common-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-rdbms-4.1.19.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/janino-3.0.16.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/algebra_2.12-2.0.0-M2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/velocity-1.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.servlet-api-4.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-jobclient-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/oro-2.0.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/htrace-core4-4.1.0-incubating.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-mapred-1.8.2-hadoop2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-datatype-jsr310-2.11.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-kvstore_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-repl_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jul-to-slf4j-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-codec-1.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-cli-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-hk2-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guice-4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javassist-3.25.0-GA.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-lang-2.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/macro-compat_2.12-1.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-io-2.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-hive-thriftserver_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-dbcp-1.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-core-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-0.23-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-encoding-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okhttp-3.12.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guice-servlet-4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-hdfs-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-daemon-1.0.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-library-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-api-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jsr305-3.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json-1.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-network-common_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zstd-jni-1.4.8-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/snappy-java-1.1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-macros_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-annotations-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-simplekdc-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-logging-1.1.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stream-2.9.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-httpclient-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-hive_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-jackson-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-format-2.4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-core-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json-smart-2.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javax.jdo-3.2.0-m3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zjsonpatch-0.3.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-pkix-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/opencsv-2.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/machinist_2.12-0.6.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aopalliance-repackaged-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-platform_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/netty-all-4.1.51.Final.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-admissionregistration-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-rbac-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/osgi-resource-locator-1.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-annotations-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-config-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/libfb303-0.9.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-unsafe_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-beanutils-1.9.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-tags_2.12-3.1.1-tests.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-llap-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mllib_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-identity-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/log4j-1.2.17.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-settings-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-collection-compat_2.12-2.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-jvm-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/derby-10.12.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-storage-api-2.7.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/JLargeArrays-1.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-api-jdo-4.2.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/JTransforms-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javolution-5.5.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/geronimo-jcache_1.0_spec-1.0-alpha-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/minlog-1.3.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-container-servlet-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/objenesis-2.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/logging-interceptor-3.12.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-compiler-3.0.16.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-certificates-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jodd-core-3.5.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-sql_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jcip-annotations-1.0-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-asn1-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-json-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.inject-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-server-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-dataformat-yaml-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-server-web-proxy-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-registry-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-catalyst_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/HikariCP-2.5.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-crypto-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/dnsjava-2.1.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/univocity-parsers-2.9.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/lz4-java-1.7.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/snakeyaml-1.24.jar":"System Classpath"}} -{"Event":"SparkListenerApplicationStart","App Name":"Rapids Spark Profiling Tool Unit Tests","App ID":"local-1622043423018","Timestamp":1622043422343,"User":"user1"} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/lib/jvm/java-8-openjdk-amd64/jre","Java Version":"1.8.0_282 (Private Build)","Scala Version":"version 2.12.10"},"Spark Properties":{"spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.scheduler.mode":"FIFO","spark.driver.memory":"30g","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423010"},"Hadoop Properties":{"hadoop.service.shutdown.timeout":"30s","yarn.resourcemanager.amlauncher.thread-count":"50","yarn.sharedcache.enabled":"false","fs.s3a.connection.maximum":"15","yarn.nodemanager.numa-awareness.numactl.cmd":"/usr/bin/numactl","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","mapreduce.framework.name":"local","yarn.sharedcache.uploader.server.thread-count":"50","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","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","hadoop.registry.secure":"false","hadoop.shell.safely.delete.limit.num.files":"100","dfs.bytes-per-checksum":"512","mapreduce.job.acl-view-job":" ","fs.s3a.s3guard.ddb.background.sleep":"25ms","fs.s3a.retry.limit":"${fs.s3a.attempts.maximum}","mapreduce.jobhistory.loadedjobs.cache.size":"5","fs.s3a.s3guard.ddb.table.create":"false","yarn.nodemanager.amrmproxy.enabled":"false","yarn.timeline-service.entity-group-fs-store.with-user-dir":"false","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.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.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","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","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.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.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","yarn.timeline-service.client.fd-clean-interval-secs":"60","yarn.resourcemanager.nodemanagers.heartbeat-interval-ms":"1000","hadoop.common.configuration.version":"3.0.0","fs.s3a.s3guard.ddb.table.capacity.read":"500","yarn.nodemanager.remote-app-log-dir-suffix":"logs","yarn.nodemanager.windows-container.cpu-limit.enabled":"false","yarn.nodemanager.runtime.linux.docker.privileged-containers.allowed":"false","file.blocksize":"67108864","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","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","fs.har.impl.disable.cache":"true","yarn.webapp.ui2.enable":"false","io.compression.codec.bzip2.library":"system-native","yarn.nodemanager.distributed-scheduling.enabled":"false","mapreduce.shuffle.connection-keep-alive.timeout":"5","yarn.resourcemanager.webapp.https.address":"${yarn.resourcemanager.hostname}:8090","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.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","fs.ftp.transfer.mode":"BLOCK_TRANSFER_MODE","ipc.server.log.slow.rpc":"false","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","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","hadoop.security.credential.clear-text-fallback":"true","yarn.nodemanager.collector-service.thread-count":"5","fs.azure.secure.mode":"false","mapreduce.jobhistory.joblist.cache.size":"20000","fs.ftp.host":"0.0.0.0","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":"false","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","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","yarn.client.load.resource-types.from-server":"false","ha.zookeeper.session-timeout.ms":"10000","tfile.io.chunk.size":"1048576","fs.s3a.s3guard.ddb.table.capacity.write":"100","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","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","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","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","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","mapreduce.job.max.split.locations":"15","yarn.resourcemanager.nm-container-queuing.max-queue-length":"15","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.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","fs.s3a.buffer.dir":"${hadoop.tmp.dir}/s3a","hadoop.ssl.enabled.protocols":"TLSv1,SSLv2Hello,TLSv1.1,TLSv1.2","mapreduce.jobhistory.admin.address":"0.0.0.0:10033","yarn.log-aggregation-status.time-out.ms":"600000","fs.s3a.assumed.role.sts.endpoint.region":"us-west-1","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.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","fs.azure.local.sas.key.mode":"false","ipc.maximum.data.length":"67108864","mapreduce.shuffle.max.threads":"0","yarn.router.pipeline.cache-max-size":"25","yarn.resourcemanager.nm-container-queuing.load-comparator":"QUEUE_LENGTH","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","mapreduce.jobhistory.http.policy":"HTTP_ONLY","yarn.sharedcache.store.in-memory.check-period-mins":"720","hadoop.security.group.mapping.ldap.ssl":"false","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.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.nodemanager.resourcemanager.minimum.version":"NONE","mapreduce.job.speculative.speculative-cap-running-tasks":"0.1","yarn.admin.acl":"*","yarn.nodemanager.recovery.supervised":"false","yarn.sharedcache.admin.thread-count":"1","yarn.resourcemanager.ha.automatic-failover.enabled":"true","mapreduce.reduce.skip.maxgroups":"0","mapreduce.reduce.shuffle.connect.timeout":"180000","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","hadoop.ssl.enabled":"false","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":"10","mapreduce.reduce.shuffle.input.buffer.percent":"0.70","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.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":"2147483647","yarn.resourcemanager.webapp.xfs-filter.xframe-options":"SAMEORIGIN","mapreduce.jobhistory.done-dir":"${yarn.app.mapreduce.am.staging-dir}/history/done","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","yarn.webapp.filter-entity-list-by-user":"false","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}","hadoop.workaround.non.threadsafe.getpwuid":"true","fs.df.interval":"60000","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","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","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.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","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.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","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","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.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","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","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":"${fs.s3a.attempts.maximum}","hadoop.http.authentication.type":"simple","mapreduce.job.cache.limit.max-resources":"0","mapreduce.task.userlog.limit.kb":"0","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","hadoop.user.group.static.mapping.overrides":"dr.who=;","fs.azure.sas.expiry.period":"90d","mapreduce.jobhistory.recovery.store.class":"org.apache.hadoop.mapreduce.v2.hs.HistoryServerFileSystemStateStoreService","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","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","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":"1000ms","seq.io.sort.factor":"100","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/","mapreduce.task.profile":"false","yarn.resourcemanager.fs.state-store.uri":"${hadoop.tmp.dir}/yarn/system/rmstore","mapreduce.jobhistory.always-scan-user-dir":"false","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","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","fs.automatic.close":"true","yarn.nodemanager.hostname":"0.0.0.0","yarn.nodemanager.resource.memory.cgroups.swappiness":"0","ftp.stream-buffer-size":"4096","yarn.fail-fast":"false","yarn.timeline-service.app-aggregation-interval-secs":"15","hadoop.security.group.mapping.ldap.search.filter.user":"(&(objectClass=user)(sAMAccountName={0}))","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","yarn.resourcemanager.delegation.token.max-lifetime":"*********(redacted)","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.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.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.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","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","hadoop.security.dns.log-slow-lookups.enabled":"false","mapreduce.job.committer.setup.cleanup.needed":"true","mapreduce.job.running.reduce.limit":"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","fs.s3a.max.total.tasks":"5","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.nodemanager.health-checker.script.timeout-ms":"1200000","yarn.timeline-service.leveldb-timeline-store.start-time-write-cache-size":"10000","yarn.resourcemanager.resource-profiles.enabled":"false","yarn.timeline-service.hbase-schema.prefix":"prod.","fs.azure.authorization":"false","mapreduce.map.log.level":"INFO","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","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","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","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":"0,1","yarn.timeline-service.client.internal-timers-ttl-secs":"420","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","ipc.client.tcpnodelay":"true","ipc.client.rpc-timeout.ms":"0","yarn.nodemanager.webapp.rest-csrf.methods-to-ignore":"GET,OPTIONS,HEAD","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","fs.s3a.socket.recv.buffer":"8192","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.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","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","mapreduce.shuffle.ssl.enabled":"false","yarn.timeline-service.hbase.coprocessor.app-final-value-retention-milliseconds":"259200000","fs.s3a.committer.staging.abort.pending.uploads":"true","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","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":"100M","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","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","yarn.resourcemanager.ha.enabled":"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.federation.subcluster-resolver.class":"org.apache.hadoop.yarn.server.federation.resolver.DefaultSubClusterResolverImpl","yarn.resourcemanager.zk-state-store.parent-path":"/rmstore","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":"fail","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","fs.AbstractFileSystem.adl.impl":"org.apache.hadoop.fs.adl.Adl","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":"128","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.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.federation.registry.base-dir":"yarnfederation/","mapreduce.job.max.map":"-1","mapreduce.job.local-fs.single-disk-limit.bytes":"-1","mapreduce.job.ubertask.maxreduces":"1","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","yarn.client.nodemanager-connect.retry-interval-ms":"10000","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","yarn.timeline-service.webapp.rest-csrf.enabled":"false"},"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":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Private Build","java.vm.specification.version":"1.8","user.home":"/home/user1","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/amd64","user.dir":"/home/user1/workspace/spark-rapids-another","java.library.path":"/usr/local/cuda/lib64:/usr/java/packages/lib/amd64:/usr/lib/x86_64-linux-gnu/jni:/lib/x86_64-linux-gnu:/usr/lib/x86_64-linux-gnu:/usr/lib/jni:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.282-b08","jetty.git.hash":"e46af88704a893fc12cb0e3bf46e2c7b48a009e7","java.endorsed.dirs":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/endorsed","java.runtime.version":"1.8.0_282-8u282-b08-0ubuntu1~18.04-b08","java.vm.info":"mixed mode","java.ext.dirs":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"OpenJDK Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/resources.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/rt.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/sunrsasign.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jsse.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jce.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/charsets.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jfr.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"4.15.0-130-generic","sun.os.patch.level":"unknown","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","user.language":"en","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.print.PSPrinterJob","java.awt.graphicsenv":"sun.awt.X11GraphicsEnvironment","awt.toolkit":"sun.awt.X11.XToolkit","os.name":"Linux","java.vm.vendor":"Private Build","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"user1","java.vm.name":"OpenJDK 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master local[1] --conf spark.driver.memory=30g --class com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils --jars /home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar /home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar dsAndDf /home/user1/testeventlogDir 100001","java.home":"/usr/lib/jvm/java-8-openjdk-amd64/jre","java.version":"1.8.0_282","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-exec-2.3.7-core.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aopalliance-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/breeze-macros_2.12-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-yarn_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jaxb-api-2.2.11.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-scala_2.12-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okhttp-2.7.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-scheduling-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-jaxrs-json-provider-2.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-storageclass-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-sketch_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/flatbuffers-java-1.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-media-jaxb-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/chill_2.12-0.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-policy-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guava-14.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jline-2.14.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stax-api-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/chill-java-0.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/shims-0.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jdo-api-3.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-autoscaling-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/gson-2.2.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/py4j-0.10.9.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-core_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-memory-netty-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-jaxb-annotations-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-kubernetes_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-admin-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/slf4j-api-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stax2-api-3.1.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-core_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-xdr-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-common-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-apiextensions-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/antlr4-runtime-4.8-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-common-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/antlr-runtime-3.5.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-core-4.1.17.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mesos_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-streaming_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jsp-api-2.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-framework-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/activation-1.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/mesos-1.4.0-shaded-protobuf.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/pyrolite-4.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/libthrift-0.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-ipc-1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-scheduler-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.annotation-api-1.3.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-client-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-crypto-1.1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-util-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-events-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-databind-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/xbean-asm7-shaded-4.15.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-mapreduce-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/core-1.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okio-1.14.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-discovery-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zookeeper-3.4.14.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/woodstox-core-5.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.xml.bind-api-2.3.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-apps-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-service-rpc-3.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mllib-local_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-launcher_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/httpcore-4.4.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jcl-over-slf4j-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-tags_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-jdbc-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-client-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arpack_combined_all-0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/token-provider-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/joda-time-2.10.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-graphx_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-format-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/re2j-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kryo-shaded-4.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jpam-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-hadoop-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/super-csv-2.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/breeze_2.12-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-jackson_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-collections-3.2.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-core-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-beeline-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-ast_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-xml_2.12-1.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-server-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/bonecp-0.8.0.RELEASE.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/slf4j-log4j12-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-shims-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/leveldbjni-all-1.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-reflect-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-serde-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/audience-annotations-0.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-batch-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-coordination-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-common-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-paranamer-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/threeten-extra-1.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-client-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/compress-lzf-1.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javax.inject-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/conf/":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-core-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/accessors-smart-1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-math3-3.4.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/RoaringBitmap-0.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ehcache-3.3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-column-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.activation-api-1.2.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-configuration2-2.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-api-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-recipes-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-network-shuffle_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/xz-1.5.jar":"System Classpath","spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar":"Added By User","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/dropwizard-metrics-hadoop-metrics2-reporter-0.1.2.jar":"System Classpath","spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar":"Added By User","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aircompressor-0.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-graphite-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-jmx-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-server-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/protobuf-java-2.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-util_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-lang3-3.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-memory-core-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-vector-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/generex-1.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/httpclient-4.5.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-client-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-compress-1.20.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-metastore-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-auth-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-scalap_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.ws.rs-api-2.1.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ST4-4.0.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-locator-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-networking-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-extensions-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jaxb-runtime-2.3.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-compiler-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/shapeless_2.12-2.3.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-core-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-pool-1.5.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-metrics-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jta-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ivy-2.4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/paranamer-2.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/istack-commons-runtime-3.0.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-text-1.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-cli-1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-core-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-net-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-utils-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/transaction-api-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/automaton-1.11-8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-parser-combinators_2.12-1.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/cats-kernel_2.12-2.0.0-M4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/nimbus-jose-jwt-4.41.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-util-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-container-servlet-core-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-vector-code-gen-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.validation-api-2.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-jaxrs-base-2.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-common-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-rdbms-4.1.19.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/janino-3.0.16.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/algebra_2.12-2.0.0-M2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/velocity-1.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.servlet-api-4.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-jobclient-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/oro-2.0.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/htrace-core4-4.1.0-incubating.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-mapred-1.8.2-hadoop2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-datatype-jsr310-2.11.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-kvstore_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-repl_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jul-to-slf4j-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-codec-1.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-cli-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-hk2-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guice-4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javassist-3.25.0-GA.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-lang-2.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/macro-compat_2.12-1.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-io-2.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-hive-thriftserver_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-dbcp-1.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-core-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-0.23-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-encoding-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okhttp-3.12.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guice-servlet-4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-hdfs-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-daemon-1.0.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-library-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-api-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jsr305-3.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json-1.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-network-common_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zstd-jni-1.4.8-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/snappy-java-1.1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-macros_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-annotations-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-simplekdc-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-logging-1.1.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stream-2.9.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-httpclient-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-hive_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-jackson-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-format-2.4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-core-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json-smart-2.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javax.jdo-3.2.0-m3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zjsonpatch-0.3.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-pkix-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/opencsv-2.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/machinist_2.12-0.6.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aopalliance-repackaged-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-platform_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/netty-all-4.1.51.Final.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-admissionregistration-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-rbac-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/osgi-resource-locator-1.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-annotations-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-config-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/libfb303-0.9.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-unsafe_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-beanutils-1.9.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-tags_2.12-3.1.1-tests.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-llap-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mllib_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-identity-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/log4j-1.2.17.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-settings-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-collection-compat_2.12-2.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-jvm-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/derby-10.12.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-storage-api-2.7.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/JLargeArrays-1.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-api-jdo-4.2.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/JTransforms-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javolution-5.5.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/geronimo-jcache_1.0_spec-1.0-alpha-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/minlog-1.3.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-container-servlet-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/objenesis-2.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/logging-interceptor-3.12.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-compiler-3.0.16.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-certificates-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jodd-core-3.5.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-sql_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jcip-annotations-1.0-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-asn1-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-json-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.inject-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-server-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-dataformat-yaml-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-server-web-proxy-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-registry-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-catalyst_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/HikariCP-2.5.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-crypto-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/dnsjava-2.1.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/univocity-parsers-2.9.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/lz4-java-1.7.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/snakeyaml-1.24.jar":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Rapids Spark Profiling Tool Unit Tests","App ID":"local-1622043423010","Timestamp":1622043422343,"User":"user1"} {"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":0,"description":"json at QualificationInfoUtils.scala:76","details":"org.apache.spark.sql.DataFrameWriter.json(DataFrameWriter.scala:853)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1(QualificationInfoUtils.scala:76)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1$adapted(QualificationInfoUtils.scala:73)\norg.apache.spark.rapids.TestUtils$.withTempPath(TestUtils.scala:31)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genDatasetEventLog(QualificationInfoUtils.scala:73)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:185)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nExecute InsertIntoHadoopFsRelationCommand (8)\n+- * SerializeFromObject (7)\n +- * MapElements (6)\n +- * Filter (5)\n +- * DeserializeToObject (4)\n +- * Project (3)\n +- * SerializeFromObject (2)\n +- Scan (1)\n\n\n(1) Scan\nOutput [1]: [obj#3]\nArguments: obj#3: scala.Tuple3, ParallelCollectionRDD[0] at parallelize at QualificationInfoUtils.scala:64\n\n(2) SerializeFromObject [codegen id : 1]\nInput [1]: [obj#3]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._1, true, false) AS _1#4, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._2, true, false) AS _2#5, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._3 AS _3#6]\n\n(3) Project [codegen id : 1]\nOutput [3]: [_1#4 AS name#11, _2#5 AS friend#12, _3#6 AS age#13]\nInput [3]: [_1#4, _2#5, _3#6]\n\n(4) DeserializeToObject [codegen id : 1]\nInput [3]: [name#11, friend#12, age#13]\nArguments: newInstance(class com.nvidia.spark.rapids.tool.profiling.RapidsFriends), obj#30: com.nvidia.spark.rapids.tool.profiling.RapidsFriends\n\n(5) Filter [codegen id : 1]\nInput [1]: [obj#30]\nCondition : com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$$$Lambda$1503/443638755@31f31b20.apply\n\n(6) MapElements [codegen id : 1]\nInput [1]: [obj#30]\nArguments: com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$$$Lambda$1505/1108374296@6723cce7, obj#31: scala.Tuple2\n\n(7) SerializeFromObject [codegen id : 1]\nInput [1]: [obj#31]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1, true, false) AS _1#32, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._2 AS _2#33]\n\n(8) Execute InsertIntoHadoopFsRelationCommand\nInput [2]: [_1#32, _2#33]\nArguments: file:/tmp/spark-4a9e661e-fc37-4f5f-8bbc-1647dbaebbaf, false, JSON, Map(path -> /tmp/spark-4a9e661e-fc37-4f5f-8bbc-1647dbaebbaf), ErrorIfExists, [_1, _2]\n\n","sparkPlanInfo":{"nodeName":"Execute InsertIntoHadoopFsRelationCommand","simpleString":"Execute InsertIntoHadoopFsRelationCommand file:/tmp/spark-4a9e661e-fc37-4f5f-8bbc-1647dbaebbaf, false, JSON, Map(path -> /tmp/spark-4a9e661e-fc37-4f5f-8bbc-1647dbaebbaf), ErrorIfExists, [_1, _2]","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1, true, false) AS _1#32, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._2 AS _2#33]","children":[{"nodeName":"MapElements","simpleString":"MapElements com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$$$Lambda$1505/1108374296@6723cce7, obj#31: scala.Tuple2","children":[{"nodeName":"Filter","simpleString":"Filter com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$$$Lambda$1503/443638755@31f31b20.apply","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject newInstance(class com.nvidia.spark.rapids.tool.profiling.RapidsFriends), obj#30: com.nvidia.spark.rapids.tool.profiling.RapidsFriends","children":[{"nodeName":"Project","simpleString":"Project [_1#4 AS name#11, _2#5 AS friend#12, _3#6 AS age#13]","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._1, true, false) AS _1#4, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._2, true, false) AS _2#5, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._3 AS _3#6]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Scan","simpleString":"Scan[obj#3]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":6,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":5,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":4,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of written files","accumulatorId":0,"metricType":"sum"},{"name":"written output","accumulatorId":1,"metricType":"size"},{"name":"number of output rows","accumulatorId":2,"metricType":"sum"},{"name":"number of dynamic part","accumulatorId":3,"metricType":"sum"}]},"time":1622043425909} -{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1622043426367,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"json at QualificationInfoUtils.scala:76","Number of Tasks":12,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"Scan\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at QualificationInfoUtils.scala:64","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameWriter.json(DataFrameWriter.scala:853)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1(QualificationInfoUtils.scala:76)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1$adapted(QualificationInfoUtils.scala:73)\norg.apache.spark.rapids.TestUtils$.withTempPath(TestUtils.scala:31)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genDatasetEventLog(QualificationInfoUtils.scala:73)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:185)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[0],"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"1\",\"name\":\"Execute InsertIntoHadoopFsRelationCommand\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"0","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"json at QualificationInfoUtils.scala:76","Number of Tasks":12,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"Scan\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at QualificationInfoUtils.scala:64","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameWriter.json(DataFrameWriter.scala:853)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1(QualificationInfoUtils.scala:76)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1$adapted(QualificationInfoUtils.scala:73)\norg.apache.spark.rapids.TestUtils$.withTempPath(TestUtils.scala:31)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genDatasetEventLog(QualificationInfoUtils.scala:73)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:185)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043426387,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"1\",\"name\":\"Execute InsertIntoHadoopFsRelationCommand\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"0","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"}} +{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1622043426367,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"json at QualificationInfoUtils.scala:76","Number of Tasks":12,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"Scan\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at QualificationInfoUtils.scala:64","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameWriter.json(DataFrameWriter.scala:853)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1(QualificationInfoUtils.scala:76)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1$adapted(QualificationInfoUtils.scala:73)\norg.apache.spark.rapids.TestUtils$.withTempPath(TestUtils.scala:31)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genDatasetEventLog(QualificationInfoUtils.scala:73)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:185)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[0],"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"1\",\"name\":\"Execute InsertIntoHadoopFsRelationCommand\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"0","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423010"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"json at QualificationInfoUtils.scala:76","Number of Tasks":12,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"Scan\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at QualificationInfoUtils.scala:64","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameWriter.json(DataFrameWriter.scala:853)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1(QualificationInfoUtils.scala:76)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1$adapted(QualificationInfoUtils.scala:73)\norg.apache.spark.rapids.TestUtils$.withTempPath(TestUtils.scala:31)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genDatasetEventLog(QualificationInfoUtils.scala:73)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:185)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043426387,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"1\",\"name\":\"Execute InsertIntoHadoopFsRelationCommand\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"0","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423010"}} {"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1622043426546,"Executor ID":"driver","Host":"10.10.10.100","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} {"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1622043426567,"Executor ID":"driver","Host":"10.10.10.100","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} {"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1622043426568,"Executor ID":"driver","Host":"10.10.10.100","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} @@ -41,9 +41,9 @@ {"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":0,"accumUpdates":[[0,12],[1,16560],[2,720],[3,0]]} {"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":0,"time":1622043427215} {"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":1,"description":"count at QualificationInfoUtils.scala:94","details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\n* HashAggregate (15)\n+- Exchange (14)\n +- * HashAggregate (13)\n +- * Project (12)\n +- * SortMergeJoin Inner (11)\n :- * Sort (5)\n : +- Exchange (4)\n : +- * Project (3)\n : +- * SerializeFromObject (2)\n : +- Scan (1)\n +- * Sort (10)\n +- Exchange (9)\n +- * Project (8)\n +- * SerializeFromObject (7)\n +- Scan (6)\n\n\n(1) Scan\nOutput [1]: [obj#40]\nArguments: obj#40: int, ParallelCollectionRDD[5] at makeRDD at QualificationInfoUtils.scala:92\n\n(2) SerializeFromObject [codegen id : 1]\nInput [1]: [obj#40]\nArguments: [input[0, int, false] AS value#41]\n\n(3) Project [codegen id : 1]\nOutput [1]: [value#41 AS a#51]\nInput [1]: [value#41]\n\n(4) Exchange\nInput [1]: [a#51]\nArguments: hashpartitioning(a#51, 200), ENSURE_REQUIREMENTS, [id=#93]\n\n(5) Sort [codegen id : 2]\nInput [1]: [a#51]\nArguments: [a#51 ASC NULLS FIRST], false, 0\n\n(6) Scan\nOutput [1]: [obj#46]\nArguments: obj#46: int, ParallelCollectionRDD[6] at makeRDD at QualificationInfoUtils.scala:93\n\n(7) SerializeFromObject [codegen id : 3]\nInput [1]: [obj#46]\nArguments: [input[0, int, false] AS value#47]\n\n(8) Project [codegen id : 3]\nOutput [1]: [value#47 AS b#53]\nInput [1]: [value#47]\n\n(9) Exchange\nInput [1]: [b#53]\nArguments: hashpartitioning(b#53, 200), ENSURE_REQUIREMENTS, [id=#102]\n\n(10) Sort [codegen id : 4]\nInput [1]: [b#53]\nArguments: [b#53 ASC NULLS FIRST], false, 0\n\n(11) SortMergeJoin [codegen id : 5]\nLeft keys [1]: [a#51]\nRight keys [1]: [b#53]\nJoin condition: None\n\n(12) Project [codegen id : 5]\nOutput: []\nInput [2]: [a#51, b#53]\n\n(13) HashAggregate [codegen id : 5]\nInput: []\nKeys: []\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count#64L]\nResults [1]: [count#65L]\n\n(14) Exchange\nInput [1]: [count#65L]\nArguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#111]\n\n(15) HashAggregate [codegen id : 6]\nInput [1]: [count#65L]\nKeys: []\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#61L]\nResults [1]: [count(1)#61L AS count#62L]\n\n","sparkPlanInfo":{"nodeName":"WholeStageCodegen (6)","simpleString":"WholeStageCodegen (6)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Exchange","simpleString":"Exchange SinglePartition, ENSURE_REQUIREMENTS, [id=#111]","children":[{"nodeName":"WholeStageCodegen (5)","simpleString":"WholeStageCodegen (5)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[], functions=[partial_count(1)])","children":[{"nodeName":"Project","simpleString":"Project","children":[{"nodeName":"SortMergeJoin","simpleString":"SortMergeJoin [a#51], [b#53], Inner","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"Sort","simpleString":"Sort [a#51 ASC NULLS FIRST], false, 0","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(a#51, 200), ENSURE_REQUIREMENTS, [id=#93]","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [value#41 AS a#51]","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [input[0, int, false] AS value#41]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Scan","simpleString":"Scan[obj#40]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":116,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":115,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":74,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":75,"metricType":"nsTiming"},{"name":"records read","accumulatorId":72,"metricType":"sum"},{"name":"local bytes read","accumulatorId":70,"metricType":"size"},{"name":"fetch wait time","accumulatorId":71,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":68,"metricType":"size"},{"name":"local blocks read","accumulatorId":67,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":66,"metricType":"sum"},{"name":"data size","accumulatorId":65,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":69,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":73,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"sort time","accumulatorId":112,"metricType":"timing"},{"name":"peak memory","accumulatorId":113,"metricType":"size"},{"name":"spill size","accumulatorId":114,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":111,"metricType":"timing"}]}],"metadata":{},"metrics":[]},{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"Sort","simpleString":"Sort [b#53 ASC NULLS FIRST], false, 0","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(b#53, 200), ENSURE_REQUIREMENTS, [id=#102]","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"Project","simpleString":"Project [value#47 AS b#53]","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [input[0, int, false] AS value#47]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Scan","simpleString":"Scan[obj#46]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":122,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":121,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":85,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":86,"metricType":"nsTiming"},{"name":"records read","accumulatorId":83,"metricType":"sum"},{"name":"local bytes read","accumulatorId":81,"metricType":"size"},{"name":"fetch wait time","accumulatorId":82,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":79,"metricType":"size"},{"name":"local blocks read","accumulatorId":78,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":77,"metricType":"sum"},{"name":"data size","accumulatorId":76,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":80,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":84,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"sort time","accumulatorId":118,"metricType":"timing"},{"name":"peak memory","accumulatorId":119,"metricType":"size"},{"name":"spill size","accumulatorId":120,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":117,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":110,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":107,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":108,"metricType":"timing"},{"name":"peak memory","accumulatorId":106,"metricType":"size"},{"name":"number of output rows","accumulatorId":105,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":109,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":104,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":96,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":97,"metricType":"nsTiming"},{"name":"records read","accumulatorId":94,"metricType":"sum"},{"name":"local bytes read","accumulatorId":92,"metricType":"size"},{"name":"fetch wait time","accumulatorId":93,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":90,"metricType":"size"},{"name":"local blocks read","accumulatorId":89,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":88,"metricType":"sum"},{"name":"data size","accumulatorId":87,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":91,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":95,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":101,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":102,"metricType":"timing"},{"name":"peak memory","accumulatorId":100,"metricType":"size"},{"name":"number of output rows","accumulatorId":99,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":103,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":98,"metricType":"timing"}]},"time":1622043427414} -{"Event":"SparkListenerJobStart","Job ID":1,"Submission Time":1622043427637,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"43\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"15\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:93","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"39\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"14\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:92","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"29\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":200,"RDD Info":[{"RDD ID":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[18],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[14],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"20\",\"name\":\"WholeStageCodegen (5)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"35\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"25\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"20\",\"name\":\"WholeStageCodegen (5)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[11,16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[1,2],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":1,"RDD Info":[{"RDD ID":22,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"44\",\"name\":\"mapPartitionsInternal\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":21,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"16\",\"name\":\"WholeStageCodegen (6)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"19\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[3],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[1,2,3,4],"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"43\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"15\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:93","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"39\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043427644,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"14\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:92","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"29\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043427656,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"}} +{"Event":"SparkListenerJobStart","Job ID":1,"Submission Time":1622043427637,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"43\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"15\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:93","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"39\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"14\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:92","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"29\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":200,"RDD Info":[{"RDD ID":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[18],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[14],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"20\",\"name\":\"WholeStageCodegen (5)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"35\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"25\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"20\",\"name\":\"WholeStageCodegen (5)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[11,16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[1,2],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":1,"RDD Info":[{"RDD ID":22,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"44\",\"name\":\"mapPartitionsInternal\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":21,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"16\",\"name\":\"WholeStageCodegen (6)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"19\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[3],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[1,2,3,4],"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423010"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"43\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"15\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:93","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"39\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043427644,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423010"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"14\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:92","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"29\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043427656,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423010"}} {"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":0,"Attempt":0,"Launch Time":1622043427655,"Executor ID":"driver","Host":"10.10.10.100","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} {"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":1,"Attempt":0,"Launch Time":1622043427661,"Executor ID":"driver","Host":"10.10.10.100","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} {"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":2,"Attempt":0,"Launch Time":1622043427661,"Executor ID":"driver","Host":"10.10.10.100","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} diff --git a/core/src/test/resources/spark-events-qualification/cluster_information/platform/dataproc b/core/src/test/resources/spark-events-qualification/cluster_information/platform/dataproc index 114f2b77a..bf47a7e19 100644 --- a/core/src/test/resources/spark-events-qualification/cluster_information/platform/dataproc +++ b/core/src/test/resources/spark-events-qualification/cluster_information/platform/dataproc @@ -5,11 +5,11 @@ {"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"dataproc-test-w-1.c.internal","Port":46767},"Maximum Memory":11884245811,"Timestamp":1702417716594,"Maximum Onheap Memory":11884245811,"Maximum Offheap Memory":0} {"Event":"SparkListenerExecutorAdded","Timestamp":1702417716738,"Executor ID":"0","Executor Info":{"Host":"dataproc-test-w-2.c.internal","Total Cores":8,"Log Urls":{"stdout":"http://dataproc-test-w-2.c.internal:40000/logPage/?appId=app-20231212214826-0000&executorId=0&logType=stdout","stderr":"http://dataproc-test-w-2.c.internal:40000/logPage/?appId=app-20231212214826-0000&executorId=0&logType=stderr"},"Attributes":{},"Resources":{},"Resource Profile Id":0}} {"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"0","Host":"dataproc-test-w-2.c.internal","Port":43349},"Maximum Memory":11884245811,"Timestamp":1702417718124,"Maximum Onheap Memory":11884245811,"Maximum Offheap Memory":0} -{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/lib/jvm/java-8-openjdk-amd64/jre","Java Version":"1.8.0_282 (Private Build)","Scala Version":"version 2.12.10"},"Spark Properties":{"spark.driver.host":"dataproc-test-m.c.internal","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.scheduler.mode":"FIFO","spark.driver.memory":"30g","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://dataproc-test-m.c.internal:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://dataproc-test-m.c.internal:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"},"Hadoop Properties":{"hadoop.service.shutdown.timeout":"30s","yarn.resourcemanager.amlauncher.thread-count":"50","yarn.sharedcache.enabled":"false","fs.s3a.connection.maximum":"15","yarn.nodemanager.numa-awareness.numactl.cmd":"/usr/bin/numactl","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","mapreduce.framework.name":"local","yarn.sharedcache.uploader.server.thread-count":"50","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","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","hadoop.registry.secure":"false","hadoop.shell.safely.delete.limit.num.files":"100","dfs.bytes-per-checksum":"512","mapreduce.job.acl-view-job":" ","fs.s3a.s3guard.ddb.background.sleep":"25ms","fs.s3a.retry.limit":"${fs.s3a.attempts.maximum}","mapreduce.jobhistory.loadedjobs.cache.size":"5","fs.s3a.s3guard.ddb.table.create":"false","yarn.nodemanager.amrmproxy.enabled":"false","yarn.timeline-service.entity-group-fs-store.with-user-dir":"false","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.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.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","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","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.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.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","yarn.timeline-service.client.fd-clean-interval-secs":"60","yarn.resourcemanager.nodemanagers.heartbeat-interval-ms":"1000","hadoop.common.configuration.version":"3.0.0","fs.s3a.s3guard.ddb.table.capacity.read":"500","yarn.nodemanager.remote-app-log-dir-suffix":"logs","yarn.nodemanager.windows-container.cpu-limit.enabled":"false","yarn.nodemanager.runtime.linux.docker.privileged-containers.allowed":"false","file.blocksize":"67108864","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","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","fs.har.impl.disable.cache":"true","yarn.webapp.ui2.enable":"false","io.compression.codec.bzip2.library":"system-native","yarn.nodemanager.distributed-scheduling.enabled":"false","mapreduce.shuffle.connection-keep-alive.timeout":"5","yarn.resourcemanager.webapp.https.address":"${yarn.resourcemanager.hostname}:8090","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.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","fs.ftp.transfer.mode":"BLOCK_TRANSFER_MODE","ipc.server.log.slow.rpc":"false","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","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","hadoop.security.credential.clear-text-fallback":"true","yarn.nodemanager.collector-service.thread-count":"5","fs.azure.secure.mode":"false","mapreduce.jobhistory.joblist.cache.size":"20000","fs.ftp.host":"0.0.0.0","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":"false","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","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","yarn.client.load.resource-types.from-server":"false","ha.zookeeper.session-timeout.ms":"10000","tfile.io.chunk.size":"1048576","fs.s3a.s3guard.ddb.table.capacity.write":"100","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","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","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","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","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","mapreduce.job.max.split.locations":"15","yarn.resourcemanager.nm-container-queuing.max-queue-length":"15","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.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","fs.s3a.buffer.dir":"${hadoop.tmp.dir}/s3a","hadoop.ssl.enabled.protocols":"TLSv1,SSLv2Hello,TLSv1.1,TLSv1.2","mapreduce.jobhistory.admin.address":"0.0.0.0:10033","yarn.log-aggregation-status.time-out.ms":"600000","fs.s3a.assumed.role.sts.endpoint.region":"us-west-1","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.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","fs.azure.local.sas.key.mode":"false","ipc.maximum.data.length":"67108864","mapreduce.shuffle.max.threads":"0","yarn.router.pipeline.cache-max-size":"25","yarn.resourcemanager.nm-container-queuing.load-comparator":"QUEUE_LENGTH","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","mapreduce.jobhistory.http.policy":"HTTP_ONLY","yarn.sharedcache.store.in-memory.check-period-mins":"720","hadoop.security.group.mapping.ldap.ssl":"false","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.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.nodemanager.resourcemanager.minimum.version":"NONE","mapreduce.job.speculative.speculative-cap-running-tasks":"0.1","yarn.admin.acl":"*","yarn.nodemanager.recovery.supervised":"false","yarn.sharedcache.admin.thread-count":"1","yarn.resourcemanager.ha.automatic-failover.enabled":"true","mapreduce.reduce.skip.maxgroups":"0","mapreduce.reduce.shuffle.connect.timeout":"180000","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","hadoop.ssl.enabled":"false","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":"10","mapreduce.reduce.shuffle.input.buffer.percent":"0.70","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.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":"2147483647","yarn.resourcemanager.webapp.xfs-filter.xframe-options":"SAMEORIGIN","mapreduce.jobhistory.done-dir":"${yarn.app.mapreduce.am.staging-dir}/history/done","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","yarn.webapp.filter-entity-list-by-user":"false","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}","hadoop.workaround.non.threadsafe.getpwuid":"true","fs.df.interval":"60000","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","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","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.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","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.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","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","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.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","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","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":"${fs.s3a.attempts.maximum}","hadoop.http.authentication.type":"simple","mapreduce.job.cache.limit.max-resources":"0","mapreduce.task.userlog.limit.kb":"0","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","hadoop.user.group.static.mapping.overrides":"dr.who=;","fs.azure.sas.expiry.period":"90d","mapreduce.jobhistory.recovery.store.class":"org.apache.hadoop.mapreduce.v2.hs.HistoryServerFileSystemStateStoreService","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","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","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":"1000ms","seq.io.sort.factor":"100","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/","mapreduce.task.profile":"false","yarn.resourcemanager.fs.state-store.uri":"${hadoop.tmp.dir}/yarn/system/rmstore","mapreduce.jobhistory.always-scan-user-dir":"false","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","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","fs.automatic.close":"true","yarn.nodemanager.hostname":"0.0.0.0","yarn.nodemanager.resource.memory.cgroups.swappiness":"0","ftp.stream-buffer-size":"4096","yarn.fail-fast":"false","yarn.timeline-service.app-aggregation-interval-secs":"15","hadoop.security.group.mapping.ldap.search.filter.user":"(&(objectClass=user)(sAMAccountName={0}))","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","yarn.resourcemanager.delegation.token.max-lifetime":"*********(redacted)","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.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.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.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","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","hadoop.security.dns.log-slow-lookups.enabled":"false","mapreduce.job.committer.setup.cleanup.needed":"true","mapreduce.job.running.reduce.limit":"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","fs.s3a.max.total.tasks":"5","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.nodemanager.health-checker.script.timeout-ms":"1200000","yarn.timeline-service.leveldb-timeline-store.start-time-write-cache-size":"10000","yarn.resourcemanager.resource-profiles.enabled":"false","yarn.timeline-service.hbase-schema.prefix":"prod.","fs.azure.authorization":"false","mapreduce.map.log.level":"INFO","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","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","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","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":"0,1","yarn.timeline-service.client.internal-timers-ttl-secs":"420","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","ipc.client.tcpnodelay":"true","ipc.client.rpc-timeout.ms":"0","yarn.nodemanager.webapp.rest-csrf.methods-to-ignore":"GET,OPTIONS,HEAD","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","fs.s3a.socket.recv.buffer":"8192","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.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","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","mapreduce.shuffle.ssl.enabled":"false","yarn.timeline-service.hbase.coprocessor.app-final-value-retention-milliseconds":"259200000","fs.s3a.committer.staging.abort.pending.uploads":"true","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","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":"100M","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","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","yarn.resourcemanager.ha.enabled":"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.federation.subcluster-resolver.class":"org.apache.hadoop.yarn.server.federation.resolver.DefaultSubClusterResolverImpl","yarn.resourcemanager.zk-state-store.parent-path":"/rmstore","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":"fail","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","fs.AbstractFileSystem.adl.impl":"org.apache.hadoop.fs.adl.Adl","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":"128","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.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.federation.registry.base-dir":"yarnfederation/","mapreduce.job.max.map":"-1","mapreduce.job.local-fs.single-disk-limit.bytes":"-1","mapreduce.job.ubertask.maxreduces":"1","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","yarn.client.nodemanager-connect.retry-interval-ms":"10000","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","yarn.timeline-service.webapp.rest-csrf.enabled":"false"},"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":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Private Build","java.vm.specification.version":"1.8","user.home":"/home/user1","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/amd64","user.dir":"/home/user1/workspace/spark-rapids-another","java.library.path":"/usr/local/cuda/lib64:/usr/java/packages/lib/amd64:/usr/lib/x86_64-linux-gnu/jni:/lib/x86_64-linux-gnu:/usr/lib/x86_64-linux-gnu:/usr/lib/jni:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.282-b08","jetty.git.hash":"e46af88704a893fc12cb0e3bf46e2c7b48a009e7","java.endorsed.dirs":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/endorsed","java.runtime.version":"1.8.0_282-8u282-b08-0ubuntu1~18.04-b08","java.vm.info":"mixed mode","java.ext.dirs":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"OpenJDK Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/resources.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/rt.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/sunrsasign.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jsse.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jce.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/charsets.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jfr.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"4.15.0-130-generic","sun.os.patch.level":"unknown","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","user.language":"en","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.print.PSPrinterJob","java.awt.graphicsenv":"sun.awt.X11GraphicsEnvironment","awt.toolkit":"sun.awt.X11.XToolkit","os.name":"Linux","java.vm.vendor":"Private Build","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"user1","java.vm.name":"OpenJDK 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master local[1] --conf spark.driver.memory=30g --class com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils --jars /home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar /home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar dsAndDf /home/user1/testeventlogDir 100001","java.home":"/usr/lib/jvm/java-8-openjdk-amd64/jre","java.version":"1.8.0_282","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-exec-2.3.7-core.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aopalliance-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/breeze-macros_2.12-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-yarn_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jaxb-api-2.2.11.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-scala_2.12-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okhttp-2.7.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-scheduling-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-jaxrs-json-provider-2.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-storageclass-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-sketch_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/flatbuffers-java-1.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-media-jaxb-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/chill_2.12-0.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-policy-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guava-14.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jline-2.14.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stax-api-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/chill-java-0.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/shims-0.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jdo-api-3.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-autoscaling-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/gson-2.2.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/py4j-0.10.9.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-core_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-memory-netty-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-jaxb-annotations-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-kubernetes_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-admin-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/slf4j-api-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stax2-api-3.1.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-core_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-xdr-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-common-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-apiextensions-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/antlr4-runtime-4.8-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-common-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/antlr-runtime-3.5.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-core-4.1.17.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mesos_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-streaming_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jsp-api-2.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-framework-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/activation-1.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/mesos-1.4.0-shaded-protobuf.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/pyrolite-4.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/libthrift-0.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-ipc-1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-scheduler-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.annotation-api-1.3.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-client-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-crypto-1.1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-util-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-events-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-databind-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/xbean-asm7-shaded-4.15.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-mapreduce-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/core-1.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okio-1.14.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-discovery-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zookeeper-3.4.14.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/woodstox-core-5.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.xml.bind-api-2.3.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-apps-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-service-rpc-3.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mllib-local_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-launcher_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/httpcore-4.4.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jcl-over-slf4j-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-tags_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-jdbc-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-client-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arpack_combined_all-0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/token-provider-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/joda-time-2.10.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-graphx_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-format-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/re2j-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kryo-shaded-4.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jpam-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-hadoop-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/super-csv-2.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/breeze_2.12-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-jackson_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-collections-3.2.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-core-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-beeline-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-ast_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-xml_2.12-1.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-server-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/bonecp-0.8.0.RELEASE.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/slf4j-log4j12-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-shims-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/leveldbjni-all-1.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-reflect-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-serde-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/audience-annotations-0.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-batch-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-coordination-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-common-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-paranamer-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/threeten-extra-1.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-client-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/compress-lzf-1.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javax.inject-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/conf/":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-core-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/accessors-smart-1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-math3-3.4.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/RoaringBitmap-0.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ehcache-3.3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-column-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.activation-api-1.2.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-configuration2-2.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-api-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-recipes-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-network-shuffle_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/xz-1.5.jar":"System Classpath","spark://dataproc-test-m.c.internal:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar":"Added By User","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/dropwizard-metrics-hadoop-metrics2-reporter-0.1.2.jar":"System Classpath","spark://dataproc-test-m.c.internal:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar":"Added By User","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aircompressor-0.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-graphite-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-jmx-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-server-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/protobuf-java-2.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-util_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-lang3-3.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-memory-core-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-vector-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/generex-1.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/httpclient-4.5.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-client-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-compress-1.20.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-metastore-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-auth-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-scalap_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.ws.rs-api-2.1.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ST4-4.0.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-locator-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-networking-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-extensions-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jaxb-runtime-2.3.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-compiler-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/shapeless_2.12-2.3.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-core-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-pool-1.5.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-metrics-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jta-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ivy-2.4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/paranamer-2.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/istack-commons-runtime-3.0.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-text-1.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-cli-1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-core-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-net-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-utils-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/transaction-api-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/automaton-1.11-8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-parser-combinators_2.12-1.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/cats-kernel_2.12-2.0.0-M4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/nimbus-jose-jwt-4.41.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-util-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-container-servlet-core-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-vector-code-gen-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.validation-api-2.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-jaxrs-base-2.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-common-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-rdbms-4.1.19.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/janino-3.0.16.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/algebra_2.12-2.0.0-M2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/velocity-1.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.servlet-api-4.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-jobclient-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/oro-2.0.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/htrace-core4-4.1.0-incubating.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-mapred-1.8.2-hadoop2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-datatype-jsr310-2.11.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-kvstore_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-repl_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jul-to-slf4j-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-codec-1.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-cli-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-hk2-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guice-4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javassist-3.25.0-GA.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-lang-2.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/macro-compat_2.12-1.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-io-2.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-hive-thriftserver_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-dbcp-1.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-core-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-0.23-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-encoding-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okhttp-3.12.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guice-servlet-4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-hdfs-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-daemon-1.0.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-library-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-api-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jsr305-3.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json-1.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-network-common_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zstd-jni-1.4.8-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/snappy-java-1.1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-macros_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-annotations-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-simplekdc-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-logging-1.1.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stream-2.9.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-httpclient-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-hive_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-jackson-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-format-2.4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-core-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json-smart-2.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javax.jdo-3.2.0-m3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zjsonpatch-0.3.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-pkix-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/opencsv-2.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/machinist_2.12-0.6.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aopalliance-repackaged-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-platform_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/netty-all-4.1.51.Final.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-admissionregistration-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-rbac-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/osgi-resource-locator-1.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-annotations-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-config-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/libfb303-0.9.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-unsafe_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-beanutils-1.9.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-tags_2.12-3.1.1-tests.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-llap-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mllib_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-identity-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/log4j-1.2.17.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-settings-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-collection-compat_2.12-2.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-jvm-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/derby-10.12.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-storage-api-2.7.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/JLargeArrays-1.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-api-jdo-4.2.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/JTransforms-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javolution-5.5.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/geronimo-jcache_1.0_spec-1.0-alpha-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/minlog-1.3.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-container-servlet-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/objenesis-2.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/logging-interceptor-3.12.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-compiler-3.0.16.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-certificates-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jodd-core-3.5.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-sql_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jcip-annotations-1.0-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-asn1-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-json-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.inject-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-server-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-dataformat-yaml-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-server-web-proxy-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-registry-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-catalyst_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/HikariCP-2.5.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-crypto-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/dnsjava-2.1.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/univocity-parsers-2.9.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/lz4-java-1.7.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/snakeyaml-1.24.jar":"System Classpath"}} -{"Event":"SparkListenerApplicationStart","App Name":"Rapids Spark Profiling Tool Unit Tests","App ID":"local-1622043423018","Timestamp":1622043422343,"User":"user1"} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/lib/jvm/java-8-openjdk-amd64/jre","Java Version":"1.8.0_282 (Private Build)","Scala Version":"version 2.12.10"},"Spark Properties":{"spark.driver.host":"dataproc-test-m.c.internal","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.scheduler.mode":"FIFO","spark.driver.memory":"30g","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://dataproc-test-m.c.internal:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://dataproc-test-m.c.internal:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423016"},"Hadoop Properties":{"hadoop.service.shutdown.timeout":"30s","yarn.resourcemanager.amlauncher.thread-count":"50","yarn.sharedcache.enabled":"false","fs.s3a.connection.maximum":"15","yarn.nodemanager.numa-awareness.numactl.cmd":"/usr/bin/numactl","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","mapreduce.framework.name":"local","yarn.sharedcache.uploader.server.thread-count":"50","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","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","hadoop.registry.secure":"false","hadoop.shell.safely.delete.limit.num.files":"100","dfs.bytes-per-checksum":"512","mapreduce.job.acl-view-job":" ","fs.s3a.s3guard.ddb.background.sleep":"25ms","fs.s3a.retry.limit":"${fs.s3a.attempts.maximum}","mapreduce.jobhistory.loadedjobs.cache.size":"5","fs.s3a.s3guard.ddb.table.create":"false","yarn.nodemanager.amrmproxy.enabled":"false","yarn.timeline-service.entity-group-fs-store.with-user-dir":"false","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.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.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","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","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.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.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","yarn.timeline-service.client.fd-clean-interval-secs":"60","yarn.resourcemanager.nodemanagers.heartbeat-interval-ms":"1000","hadoop.common.configuration.version":"3.0.0","fs.s3a.s3guard.ddb.table.capacity.read":"500","yarn.nodemanager.remote-app-log-dir-suffix":"logs","yarn.nodemanager.windows-container.cpu-limit.enabled":"false","yarn.nodemanager.runtime.linux.docker.privileged-containers.allowed":"false","file.blocksize":"67108864","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","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","fs.har.impl.disable.cache":"true","yarn.webapp.ui2.enable":"false","io.compression.codec.bzip2.library":"system-native","yarn.nodemanager.distributed-scheduling.enabled":"false","mapreduce.shuffle.connection-keep-alive.timeout":"5","yarn.resourcemanager.webapp.https.address":"${yarn.resourcemanager.hostname}:8090","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.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","fs.ftp.transfer.mode":"BLOCK_TRANSFER_MODE","ipc.server.log.slow.rpc":"false","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","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","hadoop.security.credential.clear-text-fallback":"true","yarn.nodemanager.collector-service.thread-count":"5","fs.azure.secure.mode":"false","mapreduce.jobhistory.joblist.cache.size":"20000","fs.ftp.host":"0.0.0.0","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":"false","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","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","yarn.client.load.resource-types.from-server":"false","ha.zookeeper.session-timeout.ms":"10000","tfile.io.chunk.size":"1048576","fs.s3a.s3guard.ddb.table.capacity.write":"100","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","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","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","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","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","mapreduce.job.max.split.locations":"15","yarn.resourcemanager.nm-container-queuing.max-queue-length":"15","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.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","fs.s3a.buffer.dir":"${hadoop.tmp.dir}/s3a","hadoop.ssl.enabled.protocols":"TLSv1,SSLv2Hello,TLSv1.1,TLSv1.2","mapreduce.jobhistory.admin.address":"0.0.0.0:10033","yarn.log-aggregation-status.time-out.ms":"600000","fs.s3a.assumed.role.sts.endpoint.region":"us-west-1","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.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","fs.azure.local.sas.key.mode":"false","ipc.maximum.data.length":"67108864","mapreduce.shuffle.max.threads":"0","yarn.router.pipeline.cache-max-size":"25","yarn.resourcemanager.nm-container-queuing.load-comparator":"QUEUE_LENGTH","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","mapreduce.jobhistory.http.policy":"HTTP_ONLY","yarn.sharedcache.store.in-memory.check-period-mins":"720","hadoop.security.group.mapping.ldap.ssl":"false","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.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.nodemanager.resourcemanager.minimum.version":"NONE","mapreduce.job.speculative.speculative-cap-running-tasks":"0.1","yarn.admin.acl":"*","yarn.nodemanager.recovery.supervised":"false","yarn.sharedcache.admin.thread-count":"1","yarn.resourcemanager.ha.automatic-failover.enabled":"true","mapreduce.reduce.skip.maxgroups":"0","mapreduce.reduce.shuffle.connect.timeout":"180000","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","hadoop.ssl.enabled":"false","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":"10","mapreduce.reduce.shuffle.input.buffer.percent":"0.70","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.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":"2147483647","yarn.resourcemanager.webapp.xfs-filter.xframe-options":"SAMEORIGIN","mapreduce.jobhistory.done-dir":"${yarn.app.mapreduce.am.staging-dir}/history/done","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","yarn.webapp.filter-entity-list-by-user":"false","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}","hadoop.workaround.non.threadsafe.getpwuid":"true","fs.df.interval":"60000","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","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","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.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","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.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","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","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.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","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","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":"${fs.s3a.attempts.maximum}","hadoop.http.authentication.type":"simple","mapreduce.job.cache.limit.max-resources":"0","mapreduce.task.userlog.limit.kb":"0","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","hadoop.user.group.static.mapping.overrides":"dr.who=;","fs.azure.sas.expiry.period":"90d","mapreduce.jobhistory.recovery.store.class":"org.apache.hadoop.mapreduce.v2.hs.HistoryServerFileSystemStateStoreService","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","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","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":"1000ms","seq.io.sort.factor":"100","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/","mapreduce.task.profile":"false","yarn.resourcemanager.fs.state-store.uri":"${hadoop.tmp.dir}/yarn/system/rmstore","mapreduce.jobhistory.always-scan-user-dir":"false","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","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","fs.automatic.close":"true","yarn.nodemanager.hostname":"0.0.0.0","yarn.nodemanager.resource.memory.cgroups.swappiness":"0","ftp.stream-buffer-size":"4096","yarn.fail-fast":"false","yarn.timeline-service.app-aggregation-interval-secs":"15","hadoop.security.group.mapping.ldap.search.filter.user":"(&(objectClass=user)(sAMAccountName={0}))","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","yarn.resourcemanager.delegation.token.max-lifetime":"*********(redacted)","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.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.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.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","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","hadoop.security.dns.log-slow-lookups.enabled":"false","mapreduce.job.committer.setup.cleanup.needed":"true","mapreduce.job.running.reduce.limit":"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","fs.s3a.max.total.tasks":"5","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.nodemanager.health-checker.script.timeout-ms":"1200000","yarn.timeline-service.leveldb-timeline-store.start-time-write-cache-size":"10000","yarn.resourcemanager.resource-profiles.enabled":"false","yarn.timeline-service.hbase-schema.prefix":"prod.","fs.azure.authorization":"false","mapreduce.map.log.level":"INFO","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","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","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","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":"0,1","yarn.timeline-service.client.internal-timers-ttl-secs":"420","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","ipc.client.tcpnodelay":"true","ipc.client.rpc-timeout.ms":"0","yarn.nodemanager.webapp.rest-csrf.methods-to-ignore":"GET,OPTIONS,HEAD","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","fs.s3a.socket.recv.buffer":"8192","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.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","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","mapreduce.shuffle.ssl.enabled":"false","yarn.timeline-service.hbase.coprocessor.app-final-value-retention-milliseconds":"259200000","fs.s3a.committer.staging.abort.pending.uploads":"true","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","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":"100M","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","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","yarn.resourcemanager.ha.enabled":"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.federation.subcluster-resolver.class":"org.apache.hadoop.yarn.server.federation.resolver.DefaultSubClusterResolverImpl","yarn.resourcemanager.zk-state-store.parent-path":"/rmstore","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":"fail","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","fs.AbstractFileSystem.adl.impl":"org.apache.hadoop.fs.adl.Adl","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":"128","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.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.federation.registry.base-dir":"yarnfederation/","mapreduce.job.max.map":"-1","mapreduce.job.local-fs.single-disk-limit.bytes":"-1","mapreduce.job.ubertask.maxreduces":"1","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","yarn.client.nodemanager-connect.retry-interval-ms":"10000","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","yarn.timeline-service.webapp.rest-csrf.enabled":"false"},"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":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Private Build","java.vm.specification.version":"1.8","user.home":"/home/user1","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/amd64","user.dir":"/home/user1/workspace/spark-rapids-another","java.library.path":"/usr/local/cuda/lib64:/usr/java/packages/lib/amd64:/usr/lib/x86_64-linux-gnu/jni:/lib/x86_64-linux-gnu:/usr/lib/x86_64-linux-gnu:/usr/lib/jni:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.282-b08","jetty.git.hash":"e46af88704a893fc12cb0e3bf46e2c7b48a009e7","java.endorsed.dirs":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/endorsed","java.runtime.version":"1.8.0_282-8u282-b08-0ubuntu1~18.04-b08","java.vm.info":"mixed mode","java.ext.dirs":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"OpenJDK Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/resources.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/rt.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/sunrsasign.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jsse.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jce.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/charsets.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jfr.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"4.15.0-130-generic","sun.os.patch.level":"unknown","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","user.language":"en","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.print.PSPrinterJob","java.awt.graphicsenv":"sun.awt.X11GraphicsEnvironment","awt.toolkit":"sun.awt.X11.XToolkit","os.name":"Linux","java.vm.vendor":"Private Build","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"user1","java.vm.name":"OpenJDK 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master local[1] --conf spark.driver.memory=30g --class com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils --jars /home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar /home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar dsAndDf /home/user1/testeventlogDir 100001","java.home":"/usr/lib/jvm/java-8-openjdk-amd64/jre","java.version":"1.8.0_282","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-exec-2.3.7-core.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aopalliance-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/breeze-macros_2.12-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-yarn_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jaxb-api-2.2.11.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-scala_2.12-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okhttp-2.7.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-scheduling-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-jaxrs-json-provider-2.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-storageclass-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-sketch_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/flatbuffers-java-1.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-media-jaxb-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/chill_2.12-0.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-policy-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guava-14.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jline-2.14.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stax-api-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/chill-java-0.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/shims-0.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jdo-api-3.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-autoscaling-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/gson-2.2.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/py4j-0.10.9.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-core_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-memory-netty-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-jaxb-annotations-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-kubernetes_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-admin-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/slf4j-api-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stax2-api-3.1.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-core_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-xdr-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-common-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-apiextensions-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/antlr4-runtime-4.8-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-common-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/antlr-runtime-3.5.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-core-4.1.17.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mesos_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-streaming_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jsp-api-2.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-framework-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/activation-1.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/mesos-1.4.0-shaded-protobuf.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/pyrolite-4.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/libthrift-0.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-ipc-1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-scheduler-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.annotation-api-1.3.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-client-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-crypto-1.1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-util-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-events-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-databind-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/xbean-asm7-shaded-4.15.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-mapreduce-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/core-1.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okio-1.14.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-discovery-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zookeeper-3.4.14.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/woodstox-core-5.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.xml.bind-api-2.3.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-apps-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-service-rpc-3.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mllib-local_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-launcher_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/httpcore-4.4.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jcl-over-slf4j-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-tags_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-jdbc-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-client-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arpack_combined_all-0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/token-provider-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/joda-time-2.10.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-graphx_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-format-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/re2j-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kryo-shaded-4.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jpam-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-hadoop-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/super-csv-2.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/breeze_2.12-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-jackson_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-collections-3.2.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-core-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-beeline-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-ast_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-xml_2.12-1.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-server-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/bonecp-0.8.0.RELEASE.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/slf4j-log4j12-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-shims-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/leveldbjni-all-1.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-reflect-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-serde-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/audience-annotations-0.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-batch-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-coordination-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-common-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-paranamer-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/threeten-extra-1.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-client-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/compress-lzf-1.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javax.inject-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/conf/":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-core-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/accessors-smart-1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-math3-3.4.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/RoaringBitmap-0.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ehcache-3.3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-column-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.activation-api-1.2.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-configuration2-2.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-api-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-recipes-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-network-shuffle_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/xz-1.5.jar":"System Classpath","spark://dataproc-test-m.c.internal:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar":"Added By User","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/dropwizard-metrics-hadoop-metrics2-reporter-0.1.2.jar":"System Classpath","spark://dataproc-test-m.c.internal:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar":"Added By User","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aircompressor-0.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-graphite-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-jmx-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-server-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/protobuf-java-2.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-util_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-lang3-3.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-memory-core-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-vector-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/generex-1.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/httpclient-4.5.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-client-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-compress-1.20.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-metastore-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-auth-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-scalap_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.ws.rs-api-2.1.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ST4-4.0.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-locator-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-networking-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-extensions-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jaxb-runtime-2.3.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-compiler-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/shapeless_2.12-2.3.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-core-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-pool-1.5.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-metrics-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jta-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ivy-2.4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/paranamer-2.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/istack-commons-runtime-3.0.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-text-1.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-cli-1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-core-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-net-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-utils-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/transaction-api-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/automaton-1.11-8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-parser-combinators_2.12-1.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/cats-kernel_2.12-2.0.0-M4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/nimbus-jose-jwt-4.41.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-util-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-container-servlet-core-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-vector-code-gen-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.validation-api-2.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-jaxrs-base-2.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-common-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-rdbms-4.1.19.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/janino-3.0.16.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/algebra_2.12-2.0.0-M2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/velocity-1.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.servlet-api-4.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-jobclient-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/oro-2.0.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/htrace-core4-4.1.0-incubating.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-mapred-1.8.2-hadoop2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-datatype-jsr310-2.11.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-kvstore_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-repl_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jul-to-slf4j-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-codec-1.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-cli-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-hk2-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guice-4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javassist-3.25.0-GA.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-lang-2.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/macro-compat_2.12-1.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-io-2.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-hive-thriftserver_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-dbcp-1.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-core-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-0.23-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-encoding-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okhttp-3.12.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guice-servlet-4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-hdfs-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-daemon-1.0.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-library-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-api-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jsr305-3.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json-1.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-network-common_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zstd-jni-1.4.8-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/snappy-java-1.1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-macros_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-annotations-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-simplekdc-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-logging-1.1.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stream-2.9.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-httpclient-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-hive_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-jackson-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-format-2.4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-core-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json-smart-2.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javax.jdo-3.2.0-m3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zjsonpatch-0.3.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-pkix-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/opencsv-2.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/machinist_2.12-0.6.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aopalliance-repackaged-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-platform_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/netty-all-4.1.51.Final.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-admissionregistration-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-rbac-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/osgi-resource-locator-1.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-annotations-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-config-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/libfb303-0.9.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-unsafe_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-beanutils-1.9.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-tags_2.12-3.1.1-tests.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-llap-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mllib_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-identity-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/log4j-1.2.17.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-settings-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-collection-compat_2.12-2.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-jvm-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/derby-10.12.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-storage-api-2.7.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/JLargeArrays-1.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-api-jdo-4.2.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/JTransforms-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javolution-5.5.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/geronimo-jcache_1.0_spec-1.0-alpha-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/minlog-1.3.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-container-servlet-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/objenesis-2.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/logging-interceptor-3.12.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-compiler-3.0.16.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-certificates-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jodd-core-3.5.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-sql_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jcip-annotations-1.0-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-asn1-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-json-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.inject-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-server-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-dataformat-yaml-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-server-web-proxy-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-registry-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-catalyst_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/HikariCP-2.5.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-crypto-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/dnsjava-2.1.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/univocity-parsers-2.9.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/lz4-java-1.7.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/snakeyaml-1.24.jar":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Rapids Spark Profiling Tool Unit Tests","App ID":"local-1622043423016","Timestamp":1622043422343,"User":"user1"} {"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":0,"description":"json at QualificationInfoUtils.scala:76","details":"org.apache.spark.sql.DataFrameWriter.json(DataFrameWriter.scala:853)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1(QualificationInfoUtils.scala:76)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1$adapted(QualificationInfoUtils.scala:73)\norg.apache.spark.rapids.TestUtils$.withTempPath(TestUtils.scala:31)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genDatasetEventLog(QualificationInfoUtils.scala:73)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:185)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nExecute InsertIntoHadoopFsRelationCommand (8)\n+- * SerializeFromObject (7)\n +- * MapElements (6)\n +- * Filter (5)\n +- * DeserializeToObject (4)\n +- * Project (3)\n +- * SerializeFromObject (2)\n +- Scan (1)\n\n\n(1) Scan\nOutput [1]: [obj#3]\nArguments: obj#3: scala.Tuple3, ParallelCollectionRDD[0] at parallelize at QualificationInfoUtils.scala:64\n\n(2) SerializeFromObject [codegen id : 1]\nInput [1]: [obj#3]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._1, true, false) AS _1#4, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._2, true, false) AS _2#5, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._3 AS _3#6]\n\n(3) Project [codegen id : 1]\nOutput [3]: [_1#4 AS name#11, _2#5 AS friend#12, _3#6 AS age#13]\nInput [3]: [_1#4, _2#5, _3#6]\n\n(4) DeserializeToObject [codegen id : 1]\nInput [3]: [name#11, friend#12, age#13]\nArguments: newInstance(class com.nvidia.spark.rapids.tool.profiling.RapidsFriends), obj#30: com.nvidia.spark.rapids.tool.profiling.RapidsFriends\n\n(5) Filter [codegen id : 1]\nInput [1]: [obj#30]\nCondition : com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$$$Lambda$1503/443638755@31f31b20.apply\n\n(6) MapElements [codegen id : 1]\nInput [1]: [obj#30]\nArguments: com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$$$Lambda$1505/1108374296@6723cce7, obj#31: scala.Tuple2\n\n(7) SerializeFromObject [codegen id : 1]\nInput [1]: [obj#31]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1, true, false) AS _1#32, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._2 AS _2#33]\n\n(8) Execute InsertIntoHadoopFsRelationCommand\nInput [2]: [_1#32, _2#33]\nArguments: file:/tmp/spark-4a9e661e-fc37-4f5f-8bbc-1647dbaebbaf, false, JSON, Map(path -> /tmp/spark-4a9e661e-fc37-4f5f-8bbc-1647dbaebbaf), ErrorIfExists, [_1, _2]\n\n","sparkPlanInfo":{"nodeName":"Execute InsertIntoHadoopFsRelationCommand","simpleString":"Execute InsertIntoHadoopFsRelationCommand file:/tmp/spark-4a9e661e-fc37-4f5f-8bbc-1647dbaebbaf, false, JSON, Map(path -> /tmp/spark-4a9e661e-fc37-4f5f-8bbc-1647dbaebbaf), ErrorIfExists, [_1, _2]","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1, true, false) AS _1#32, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._2 AS _2#33]","children":[{"nodeName":"MapElements","simpleString":"MapElements com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$$$Lambda$1505/1108374296@6723cce7, obj#31: scala.Tuple2","children":[{"nodeName":"Filter","simpleString":"Filter com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$$$Lambda$1503/443638755@31f31b20.apply","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject newInstance(class com.nvidia.spark.rapids.tool.profiling.RapidsFriends), obj#30: com.nvidia.spark.rapids.tool.profiling.RapidsFriends","children":[{"nodeName":"Project","simpleString":"Project [_1#4 AS name#11, _2#5 AS friend#12, _3#6 AS age#13]","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._1, true, false) AS _1#4, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._2, true, false) AS _2#5, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._3 AS _3#6]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Scan","simpleString":"Scan[obj#3]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":6,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":5,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":4,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of written files","accumulatorId":0,"metricType":"sum"},{"name":"written output","accumulatorId":1,"metricType":"size"},{"name":"number of output rows","accumulatorId":2,"metricType":"sum"},{"name":"number of dynamic part","accumulatorId":3,"metricType":"sum"}]},"time":1622043425909} -{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1622043426367,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"json at QualificationInfoUtils.scala:76","Number of Tasks":12,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"Scan\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at QualificationInfoUtils.scala:64","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameWriter.json(DataFrameWriter.scala:853)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1(QualificationInfoUtils.scala:76)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1$adapted(QualificationInfoUtils.scala:73)\norg.apache.spark.rapids.TestUtils$.withTempPath(TestUtils.scala:31)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genDatasetEventLog(QualificationInfoUtils.scala:73)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:185)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[0],"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"dataproc-test-m.c.internal","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"1\",\"name\":\"Execute InsertIntoHadoopFsRelationCommand\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://dataproc-test-m.c.internal:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://dataproc-test-m.c.internal:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"0","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"json at QualificationInfoUtils.scala:76","Number of Tasks":12,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"Scan\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at QualificationInfoUtils.scala:64","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameWriter.json(DataFrameWriter.scala:853)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1(QualificationInfoUtils.scala:76)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1$adapted(QualificationInfoUtils.scala:73)\norg.apache.spark.rapids.TestUtils$.withTempPath(TestUtils.scala:31)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genDatasetEventLog(QualificationInfoUtils.scala:73)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:185)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043426387,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"dataproc-test-m.c.internal","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"1\",\"name\":\"Execute InsertIntoHadoopFsRelationCommand\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://dataproc-test-m.c.internal:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://dataproc-test-m.c.internal:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"0","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"}} +{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1622043426367,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"json at QualificationInfoUtils.scala:76","Number of Tasks":12,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"Scan\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at QualificationInfoUtils.scala:64","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameWriter.json(DataFrameWriter.scala:853)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1(QualificationInfoUtils.scala:76)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1$adapted(QualificationInfoUtils.scala:73)\norg.apache.spark.rapids.TestUtils$.withTempPath(TestUtils.scala:31)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genDatasetEventLog(QualificationInfoUtils.scala:73)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:185)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[0],"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"dataproc-test-m.c.internal","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"1\",\"name\":\"Execute InsertIntoHadoopFsRelationCommand\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://dataproc-test-m.c.internal:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://dataproc-test-m.c.internal:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"0","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423016"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"json at QualificationInfoUtils.scala:76","Number of Tasks":12,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"Scan\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at QualificationInfoUtils.scala:64","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameWriter.json(DataFrameWriter.scala:853)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1(QualificationInfoUtils.scala:76)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1$adapted(QualificationInfoUtils.scala:73)\norg.apache.spark.rapids.TestUtils$.withTempPath(TestUtils.scala:31)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genDatasetEventLog(QualificationInfoUtils.scala:73)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:185)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043426387,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"dataproc-test-m.c.internal","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"1\",\"name\":\"Execute InsertIntoHadoopFsRelationCommand\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://dataproc-test-m.c.internal:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://dataproc-test-m.c.internal:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"0","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423016"}} {"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1622043426546,"Executor ID":"driver","Host":"dataproc-test-m.c.internal","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} {"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1622043426567,"Executor ID":"driver","Host":"dataproc-test-m.c.internal","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} {"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1622043426568,"Executor ID":"driver","Host":"dataproc-test-m.c.internal","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} @@ -39,9 +39,9 @@ {"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":0,"accumUpdates":[[0,12],[1,16560],[2,720],[3,0]]} {"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":0,"time":1622043427215} {"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":1,"description":"count at QualificationInfoUtils.scala:94","details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\n* HashAggregate (15)\n+- Exchange (14)\n +- * HashAggregate (13)\n +- * Project (12)\n +- * SortMergeJoin Inner (11)\n :- * Sort (5)\n : +- Exchange (4)\n : +- * Project (3)\n : +- * SerializeFromObject (2)\n : +- Scan (1)\n +- * Sort (10)\n +- Exchange (9)\n +- * Project (8)\n +- * SerializeFromObject (7)\n +- Scan (6)\n\n\n(1) Scan\nOutput [1]: [obj#40]\nArguments: obj#40: int, ParallelCollectionRDD[5] at makeRDD at QualificationInfoUtils.scala:92\n\n(2) SerializeFromObject [codegen id : 1]\nInput [1]: [obj#40]\nArguments: [input[0, int, false] AS value#41]\n\n(3) Project [codegen id : 1]\nOutput [1]: [value#41 AS a#51]\nInput [1]: [value#41]\n\n(4) Exchange\nInput [1]: [a#51]\nArguments: hashpartitioning(a#51, 200), ENSURE_REQUIREMENTS, [id=#93]\n\n(5) Sort [codegen id : 2]\nInput [1]: [a#51]\nArguments: [a#51 ASC NULLS FIRST], false, 0\n\n(6) Scan\nOutput [1]: [obj#46]\nArguments: obj#46: int, ParallelCollectionRDD[6] at makeRDD at QualificationInfoUtils.scala:93\n\n(7) SerializeFromObject [codegen id : 3]\nInput [1]: [obj#46]\nArguments: [input[0, int, false] AS value#47]\n\n(8) Project [codegen id : 3]\nOutput [1]: [value#47 AS b#53]\nInput [1]: [value#47]\n\n(9) Exchange\nInput [1]: [b#53]\nArguments: hashpartitioning(b#53, 200), ENSURE_REQUIREMENTS, [id=#102]\n\n(10) Sort [codegen id : 4]\nInput [1]: [b#53]\nArguments: [b#53 ASC NULLS FIRST], false, 0\n\n(11) SortMergeJoin [codegen id : 5]\nLeft keys [1]: [a#51]\nRight keys [1]: [b#53]\nJoin condition: None\n\n(12) Project [codegen id : 5]\nOutput: []\nInput [2]: [a#51, b#53]\n\n(13) HashAggregate [codegen id : 5]\nInput: []\nKeys: []\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count#64L]\nResults [1]: [count#65L]\n\n(14) Exchange\nInput [1]: [count#65L]\nArguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#111]\n\n(15) HashAggregate [codegen id : 6]\nInput [1]: [count#65L]\nKeys: []\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#61L]\nResults [1]: [count(1)#61L AS count#62L]\n\n","sparkPlanInfo":{"nodeName":"WholeStageCodegen (6)","simpleString":"WholeStageCodegen (6)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Exchange","simpleString":"Exchange SinglePartition, ENSURE_REQUIREMENTS, [id=#111]","children":[{"nodeName":"WholeStageCodegen (5)","simpleString":"WholeStageCodegen (5)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[], functions=[partial_count(1)])","children":[{"nodeName":"Project","simpleString":"Project","children":[{"nodeName":"SortMergeJoin","simpleString":"SortMergeJoin [a#51], [b#53], Inner","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"Sort","simpleString":"Sort [a#51 ASC NULLS FIRST], false, 0","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(a#51, 200), ENSURE_REQUIREMENTS, [id=#93]","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [value#41 AS a#51]","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [input[0, int, false] AS value#41]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Scan","simpleString":"Scan[obj#40]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":116,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":115,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":74,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":75,"metricType":"nsTiming"},{"name":"records read","accumulatorId":72,"metricType":"sum"},{"name":"local bytes read","accumulatorId":70,"metricType":"size"},{"name":"fetch wait time","accumulatorId":71,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":68,"metricType":"size"},{"name":"local blocks read","accumulatorId":67,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":66,"metricType":"sum"},{"name":"data size","accumulatorId":65,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":69,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":73,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"sort time","accumulatorId":112,"metricType":"timing"},{"name":"peak memory","accumulatorId":113,"metricType":"size"},{"name":"spill size","accumulatorId":114,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":111,"metricType":"timing"}]}],"metadata":{},"metrics":[]},{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"Sort","simpleString":"Sort [b#53 ASC NULLS FIRST], false, 0","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(b#53, 200), ENSURE_REQUIREMENTS, [id=#102]","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"Project","simpleString":"Project [value#47 AS b#53]","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [input[0, int, false] AS value#47]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Scan","simpleString":"Scan[obj#46]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":122,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":121,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":85,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":86,"metricType":"nsTiming"},{"name":"records read","accumulatorId":83,"metricType":"sum"},{"name":"local bytes read","accumulatorId":81,"metricType":"size"},{"name":"fetch wait time","accumulatorId":82,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":79,"metricType":"size"},{"name":"local blocks read","accumulatorId":78,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":77,"metricType":"sum"},{"name":"data size","accumulatorId":76,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":80,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":84,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"sort time","accumulatorId":118,"metricType":"timing"},{"name":"peak memory","accumulatorId":119,"metricType":"size"},{"name":"spill size","accumulatorId":120,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":117,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":110,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":107,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":108,"metricType":"timing"},{"name":"peak memory","accumulatorId":106,"metricType":"size"},{"name":"number of output rows","accumulatorId":105,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":109,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":104,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":96,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":97,"metricType":"nsTiming"},{"name":"records read","accumulatorId":94,"metricType":"sum"},{"name":"local bytes read","accumulatorId":92,"metricType":"size"},{"name":"fetch wait time","accumulatorId":93,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":90,"metricType":"size"},{"name":"local blocks read","accumulatorId":89,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":88,"metricType":"sum"},{"name":"data size","accumulatorId":87,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":91,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":95,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":101,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":102,"metricType":"timing"},{"name":"peak memory","accumulatorId":100,"metricType":"size"},{"name":"number of output rows","accumulatorId":99,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":103,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":98,"metricType":"timing"}]},"time":1622043427414} -{"Event":"SparkListenerJobStart","Job ID":1,"Submission Time":1622043427637,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"43\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"15\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:93","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"39\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"14\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:92","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"29\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":200,"RDD Info":[{"RDD ID":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[18],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[14],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"20\",\"name\":\"WholeStageCodegen (5)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"35\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"25\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"20\",\"name\":\"WholeStageCodegen (5)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[11,16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[1,2],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":1,"RDD Info":[{"RDD ID":22,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"44\",\"name\":\"mapPartitionsInternal\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":21,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"16\",\"name\":\"WholeStageCodegen (6)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"19\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[3],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[1,2,3,4],"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"dataproc-test-m.c.internal","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://dataproc-test-m.c.internal:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://dataproc-test-m.c.internal:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"43\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"15\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:93","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"39\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043427644,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"dataproc-test-m.c.internal","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://dataproc-test-m.c.internal:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://dataproc-test-m.c.internal:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"14\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:92","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"29\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043427656,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"dataproc-test-m.c.internal","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://dataproc-test-m.c.internal:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://dataproc-test-m.c.internal:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"}} +{"Event":"SparkListenerJobStart","Job ID":1,"Submission Time":1622043427637,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"43\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"15\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:93","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"39\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"14\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:92","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"29\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":200,"RDD Info":[{"RDD ID":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[18],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[14],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"20\",\"name\":\"WholeStageCodegen (5)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"35\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"25\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"20\",\"name\":\"WholeStageCodegen (5)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[11,16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[1,2],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":1,"RDD Info":[{"RDD ID":22,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"44\",\"name\":\"mapPartitionsInternal\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":21,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"16\",\"name\":\"WholeStageCodegen (6)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"19\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[3],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[1,2,3,4],"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"dataproc-test-m.c.internal","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://dataproc-test-m.c.internal:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://dataproc-test-m.c.internal:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423016"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"43\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"15\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:93","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"39\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043427644,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"dataproc-test-m.c.internal","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://dataproc-test-m.c.internal:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://dataproc-test-m.c.internal:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423016"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"14\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:92","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"29\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043427656,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"dataproc-test-m.c.internal","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://dataproc-test-m.c.internal:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://dataproc-test-m.c.internal:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423016"}} {"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":0,"Attempt":0,"Launch Time":1622043427655,"Executor ID":"driver","Host":"dataproc-test-m.c.internal","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} {"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":1,"Attempt":0,"Launch Time":1622043427661,"Executor ID":"driver","Host":"dataproc-test-m.c.internal","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} {"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":2,"Attempt":0,"Launch Time":1622043427661,"Executor ID":"driver","Host":"dataproc-test-m.c.internal","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} diff --git a/core/src/test/resources/spark-events-qualification/cluster_information/platform/emr b/core/src/test/resources/spark-events-qualification/cluster_information/platform/emr index 679a6588d..28cb0a305 100644 --- a/core/src/test/resources/spark-events-qualification/cluster_information/platform/emr +++ b/core/src/test/resources/spark-events-qualification/cluster_information/platform/emr @@ -5,11 +5,11 @@ {"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"10.59.184.228","Port":46767},"Maximum Memory":11884245811,"Timestamp":1702417716594,"Maximum Onheap Memory":11884245811,"Maximum Offheap Memory":0} {"Event":"SparkListenerExecutorAdded","Timestamp":1702417716738,"Executor ID":"0","Executor Info":{"Host":"10.59.184.244","Total Cores":8,"Log Urls":{"stdout":"http://10.59.184.244:40000/logPage/?appId=app-20231212214826-0000&executorId=0&logType=stdout","stderr":"http://10.59.184.244:40000/logPage/?appId=app-20231212214826-0000&executorId=0&logType=stderr"},"Attributes":{},"Resources":{},"Resource Profile Id":0}} {"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"0","Host":"10.59.184.244","Port":43349},"Maximum Memory":11884245811,"Timestamp":1702417718124,"Maximum Onheap Memory":11884245811,"Maximum Offheap Memory":0} -{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/lib/jvm/java-8-openjdk-amd64/jre","Java Version":"1.8.0_282 (Private Build)","Scala Version":"version 2.12.10"},"Spark Properties":{"spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.scheduler.mode":"FIFO","spark.driver.memory":"30g","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"},"Hadoop Properties":{"hadoop.service.shutdown.timeout":"30s","yarn.resourcemanager.amlauncher.thread-count":"50","yarn.sharedcache.enabled":"false","fs.s3a.connection.maximum":"15","yarn.nodemanager.numa-awareness.numactl.cmd":"/usr/bin/numactl","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","mapreduce.framework.name":"local","yarn.sharedcache.uploader.server.thread-count":"50","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","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","hadoop.registry.secure":"false","hadoop.shell.safely.delete.limit.num.files":"100","dfs.bytes-per-checksum":"512","mapreduce.job.acl-view-job":" ","fs.s3a.s3guard.ddb.background.sleep":"25ms","fs.s3a.retry.limit":"${fs.s3a.attempts.maximum}","mapreduce.jobhistory.loadedjobs.cache.size":"5","fs.s3a.s3guard.ddb.table.create":"false","yarn.nodemanager.amrmproxy.enabled":"false","yarn.timeline-service.entity-group-fs-store.with-user-dir":"false","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.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.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","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","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.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.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","yarn.timeline-service.client.fd-clean-interval-secs":"60","yarn.resourcemanager.nodemanagers.heartbeat-interval-ms":"1000","hadoop.common.configuration.version":"3.0.0","fs.s3a.s3guard.ddb.table.capacity.read":"500","yarn.nodemanager.remote-app-log-dir-suffix":"logs","yarn.nodemanager.windows-container.cpu-limit.enabled":"false","yarn.nodemanager.runtime.linux.docker.privileged-containers.allowed":"false","file.blocksize":"67108864","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","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","fs.har.impl.disable.cache":"true","yarn.webapp.ui2.enable":"false","io.compression.codec.bzip2.library":"system-native","yarn.nodemanager.distributed-scheduling.enabled":"false","mapreduce.shuffle.connection-keep-alive.timeout":"5","yarn.resourcemanager.webapp.https.address":"${yarn.resourcemanager.hostname}:8090","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.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","fs.ftp.transfer.mode":"BLOCK_TRANSFER_MODE","ipc.server.log.slow.rpc":"false","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","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","hadoop.security.credential.clear-text-fallback":"true","yarn.nodemanager.collector-service.thread-count":"5","fs.azure.secure.mode":"false","mapreduce.jobhistory.joblist.cache.size":"20000","fs.ftp.host":"0.0.0.0","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":"false","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","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","yarn.client.load.resource-types.from-server":"false","ha.zookeeper.session-timeout.ms":"10000","tfile.io.chunk.size":"1048576","fs.s3a.s3guard.ddb.table.capacity.write":"100","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","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","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","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","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","mapreduce.job.max.split.locations":"15","yarn.resourcemanager.nm-container-queuing.max-queue-length":"15","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.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","fs.s3a.buffer.dir":"${hadoop.tmp.dir}/s3a","hadoop.ssl.enabled.protocols":"TLSv1,SSLv2Hello,TLSv1.1,TLSv1.2","mapreduce.jobhistory.admin.address":"0.0.0.0:10033","yarn.log-aggregation-status.time-out.ms":"600000","fs.s3a.assumed.role.sts.endpoint.region":"us-west-1","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.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","fs.azure.local.sas.key.mode":"false","ipc.maximum.data.length":"67108864","mapreduce.shuffle.max.threads":"0","yarn.router.pipeline.cache-max-size":"25","yarn.resourcemanager.nm-container-queuing.load-comparator":"QUEUE_LENGTH","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","mapreduce.jobhistory.http.policy":"HTTP_ONLY","yarn.sharedcache.store.in-memory.check-period-mins":"720","hadoop.security.group.mapping.ldap.ssl":"false","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.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.nodemanager.resourcemanager.minimum.version":"NONE","mapreduce.job.speculative.speculative-cap-running-tasks":"0.1","yarn.admin.acl":"*","yarn.nodemanager.recovery.supervised":"false","yarn.sharedcache.admin.thread-count":"1","yarn.resourcemanager.ha.automatic-failover.enabled":"true","mapreduce.reduce.skip.maxgroups":"0","mapreduce.reduce.shuffle.connect.timeout":"180000","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","hadoop.ssl.enabled":"false","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":"10","mapreduce.reduce.shuffle.input.buffer.percent":"0.70","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.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":"2147483647","yarn.resourcemanager.webapp.xfs-filter.xframe-options":"SAMEORIGIN","mapreduce.jobhistory.done-dir":"${yarn.app.mapreduce.am.staging-dir}/history/done","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","yarn.webapp.filter-entity-list-by-user":"false","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}","hadoop.workaround.non.threadsafe.getpwuid":"true","fs.df.interval":"60000","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","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","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.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","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.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","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","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.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","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","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":"${fs.s3a.attempts.maximum}","hadoop.http.authentication.type":"simple","mapreduce.job.cache.limit.max-resources":"0","mapreduce.task.userlog.limit.kb":"0","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","hadoop.user.group.static.mapping.overrides":"dr.who=;","fs.azure.sas.expiry.period":"90d","mapreduce.jobhistory.recovery.store.class":"org.apache.hadoop.mapreduce.v2.hs.HistoryServerFileSystemStateStoreService","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","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","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":"1000ms","seq.io.sort.factor":"100","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/","mapreduce.task.profile":"false","yarn.resourcemanager.fs.state-store.uri":"${hadoop.tmp.dir}/yarn/system/rmstore","mapreduce.jobhistory.always-scan-user-dir":"false","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","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","fs.automatic.close":"true","yarn.nodemanager.hostname":"0.0.0.0","yarn.nodemanager.resource.memory.cgroups.swappiness":"0","ftp.stream-buffer-size":"4096","yarn.fail-fast":"false","yarn.timeline-service.app-aggregation-interval-secs":"15","hadoop.security.group.mapping.ldap.search.filter.user":"(&(objectClass=user)(sAMAccountName={0}))","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","yarn.resourcemanager.delegation.token.max-lifetime":"*********(redacted)","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.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.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.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","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","hadoop.security.dns.log-slow-lookups.enabled":"false","mapreduce.job.committer.setup.cleanup.needed":"true","mapreduce.job.running.reduce.limit":"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","fs.s3a.max.total.tasks":"5","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.nodemanager.health-checker.script.timeout-ms":"1200000","yarn.timeline-service.leveldb-timeline-store.start-time-write-cache-size":"10000","yarn.resourcemanager.resource-profiles.enabled":"false","yarn.timeline-service.hbase-schema.prefix":"prod.","fs.azure.authorization":"false","mapreduce.map.log.level":"INFO","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","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","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","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":"0,1","yarn.timeline-service.client.internal-timers-ttl-secs":"420","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","ipc.client.tcpnodelay":"true","ipc.client.rpc-timeout.ms":"0","yarn.nodemanager.webapp.rest-csrf.methods-to-ignore":"GET,OPTIONS,HEAD","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","fs.s3a.socket.recv.buffer":"8192","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.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","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","mapreduce.shuffle.ssl.enabled":"false","yarn.timeline-service.hbase.coprocessor.app-final-value-retention-milliseconds":"259200000","fs.s3a.committer.staging.abort.pending.uploads":"true","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","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":"100M","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","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","yarn.resourcemanager.ha.enabled":"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.federation.subcluster-resolver.class":"org.apache.hadoop.yarn.server.federation.resolver.DefaultSubClusterResolverImpl","yarn.resourcemanager.zk-state-store.parent-path":"/rmstore","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":"fail","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","fs.AbstractFileSystem.adl.impl":"org.apache.hadoop.fs.adl.Adl","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":"128","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.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.federation.registry.base-dir":"yarnfederation/","mapreduce.job.max.map":"-1","mapreduce.job.local-fs.single-disk-limit.bytes":"-1","mapreduce.job.ubertask.maxreduces":"1","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","yarn.client.nodemanager-connect.retry-interval-ms":"10000","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","yarn.timeline-service.webapp.rest-csrf.enabled":"false"},"System Properties":{"EMR_CLUSTER_ID":"j-123AB678XY321","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":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Private Build","java.vm.specification.version":"1.8","user.home":"/home/user1","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/amd64","user.dir":"/home/user1/workspace/spark-rapids-another","java.library.path":"/usr/local/cuda/lib64:/usr/java/packages/lib/amd64:/usr/lib/x86_64-linux-gnu/jni:/lib/x86_64-linux-gnu:/usr/lib/x86_64-linux-gnu:/usr/lib/jni:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.282-b08","jetty.git.hash":"e46af88704a893fc12cb0e3bf46e2c7b48a009e7","java.endorsed.dirs":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/endorsed","java.runtime.version":"1.8.0_282-8u282-b08-0ubuntu1~18.04-b08","java.vm.info":"mixed mode","java.ext.dirs":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"OpenJDK Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/resources.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/rt.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/sunrsasign.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jsse.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jce.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/charsets.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jfr.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"4.15.0-130-generic","sun.os.patch.level":"unknown","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","user.language":"en","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.print.PSPrinterJob","java.awt.graphicsenv":"sun.awt.X11GraphicsEnvironment","awt.toolkit":"sun.awt.X11.XToolkit","os.name":"Linux","java.vm.vendor":"Private Build","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"user1","java.vm.name":"OpenJDK 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master local[1] --conf spark.driver.memory=30g --class com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils --jars /home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar /home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar dsAndDf /home/user1/testeventlogDir 100001","java.home":"/usr/lib/jvm/java-8-openjdk-amd64/jre","java.version":"1.8.0_282","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-exec-2.3.7-core.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aopalliance-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/breeze-macros_2.12-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-yarn_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jaxb-api-2.2.11.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-scala_2.12-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okhttp-2.7.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-scheduling-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-jaxrs-json-provider-2.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-storageclass-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-sketch_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/flatbuffers-java-1.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-media-jaxb-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/chill_2.12-0.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-policy-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guava-14.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jline-2.14.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stax-api-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/chill-java-0.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/shims-0.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jdo-api-3.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-autoscaling-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/gson-2.2.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/py4j-0.10.9.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-core_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-memory-netty-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-jaxb-annotations-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-kubernetes_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-admin-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/slf4j-api-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stax2-api-3.1.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-core_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-xdr-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-common-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-apiextensions-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/antlr4-runtime-4.8-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-common-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/antlr-runtime-3.5.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-core-4.1.17.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mesos_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-streaming_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jsp-api-2.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-framework-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/activation-1.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/mesos-1.4.0-shaded-protobuf.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/pyrolite-4.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/libthrift-0.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-ipc-1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-scheduler-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.annotation-api-1.3.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-client-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-crypto-1.1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-util-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-events-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-databind-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/xbean-asm7-shaded-4.15.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-mapreduce-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/core-1.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okio-1.14.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-discovery-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zookeeper-3.4.14.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/woodstox-core-5.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.xml.bind-api-2.3.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-apps-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-service-rpc-3.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mllib-local_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-launcher_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/httpcore-4.4.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jcl-over-slf4j-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-tags_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-jdbc-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-client-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arpack_combined_all-0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/token-provider-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/joda-time-2.10.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-graphx_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-format-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/re2j-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kryo-shaded-4.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jpam-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-hadoop-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/super-csv-2.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/breeze_2.12-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-jackson_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-collections-3.2.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-core-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-beeline-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-ast_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-xml_2.12-1.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-server-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/bonecp-0.8.0.RELEASE.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/slf4j-log4j12-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-shims-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/leveldbjni-all-1.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-reflect-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-serde-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/audience-annotations-0.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-batch-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-coordination-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-common-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-paranamer-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/threeten-extra-1.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-client-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/compress-lzf-1.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javax.inject-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/conf/":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-core-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/accessors-smart-1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-math3-3.4.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/RoaringBitmap-0.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ehcache-3.3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-column-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.activation-api-1.2.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-configuration2-2.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-api-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-recipes-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-network-shuffle_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/xz-1.5.jar":"System Classpath","spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar":"Added By User","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/dropwizard-metrics-hadoop-metrics2-reporter-0.1.2.jar":"System Classpath","spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar":"Added By User","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aircompressor-0.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-graphite-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-jmx-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-server-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/protobuf-java-2.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-util_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-lang3-3.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-memory-core-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-vector-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/generex-1.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/httpclient-4.5.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-client-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-compress-1.20.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-metastore-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-auth-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-scalap_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.ws.rs-api-2.1.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ST4-4.0.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-locator-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-networking-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-extensions-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jaxb-runtime-2.3.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-compiler-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/shapeless_2.12-2.3.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-core-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-pool-1.5.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-metrics-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jta-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ivy-2.4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/paranamer-2.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/istack-commons-runtime-3.0.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-text-1.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-cli-1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-core-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-net-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-utils-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/transaction-api-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/automaton-1.11-8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-parser-combinators_2.12-1.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/cats-kernel_2.12-2.0.0-M4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/nimbus-jose-jwt-4.41.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-util-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-container-servlet-core-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-vector-code-gen-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.validation-api-2.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-jaxrs-base-2.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-common-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-rdbms-4.1.19.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/janino-3.0.16.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/algebra_2.12-2.0.0-M2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/velocity-1.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.servlet-api-4.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-jobclient-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/oro-2.0.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/htrace-core4-4.1.0-incubating.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-mapred-1.8.2-hadoop2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-datatype-jsr310-2.11.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-kvstore_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-repl_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jul-to-slf4j-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-codec-1.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-cli-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-hk2-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guice-4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javassist-3.25.0-GA.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-lang-2.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/macro-compat_2.12-1.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-io-2.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-hive-thriftserver_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-dbcp-1.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-core-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-0.23-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-encoding-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okhttp-3.12.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guice-servlet-4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-hdfs-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-daemon-1.0.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-library-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-api-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jsr305-3.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json-1.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-network-common_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zstd-jni-1.4.8-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/snappy-java-1.1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-macros_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-annotations-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-simplekdc-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-logging-1.1.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stream-2.9.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-httpclient-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-hive_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-jackson-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-format-2.4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-core-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json-smart-2.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javax.jdo-3.2.0-m3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zjsonpatch-0.3.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-pkix-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/opencsv-2.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/machinist_2.12-0.6.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aopalliance-repackaged-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-platform_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/netty-all-4.1.51.Final.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-admissionregistration-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-rbac-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/osgi-resource-locator-1.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-annotations-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-config-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/libfb303-0.9.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-unsafe_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-beanutils-1.9.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-tags_2.12-3.1.1-tests.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-llap-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mllib_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-identity-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/log4j-1.2.17.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-settings-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-collection-compat_2.12-2.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-jvm-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/derby-10.12.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-storage-api-2.7.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/JLargeArrays-1.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-api-jdo-4.2.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/JTransforms-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javolution-5.5.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/geronimo-jcache_1.0_spec-1.0-alpha-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/minlog-1.3.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-container-servlet-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/objenesis-2.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/logging-interceptor-3.12.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-compiler-3.0.16.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-certificates-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jodd-core-3.5.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-sql_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jcip-annotations-1.0-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-asn1-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-json-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.inject-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-server-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-dataformat-yaml-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-server-web-proxy-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-registry-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-catalyst_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/HikariCP-2.5.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-crypto-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/dnsjava-2.1.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/univocity-parsers-2.9.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/lz4-java-1.7.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/snakeyaml-1.24.jar":"System Classpath"}} -{"Event":"SparkListenerApplicationStart","App Name":"Rapids Spark Profiling Tool Unit Tests","App ID":"local-1622043423018","Timestamp":1622043422343,"User":"user1"} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/lib/jvm/java-8-openjdk-amd64/jre","Java Version":"1.8.0_282 (Private Build)","Scala Version":"version 2.12.10"},"Spark Properties":{"spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.scheduler.mode":"FIFO","spark.driver.memory":"30g","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423017"},"Hadoop Properties":{"hadoop.service.shutdown.timeout":"30s","yarn.resourcemanager.amlauncher.thread-count":"50","yarn.sharedcache.enabled":"false","fs.s3a.connection.maximum":"15","yarn.nodemanager.numa-awareness.numactl.cmd":"/usr/bin/numactl","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","mapreduce.framework.name":"local","yarn.sharedcache.uploader.server.thread-count":"50","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","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","hadoop.registry.secure":"false","hadoop.shell.safely.delete.limit.num.files":"100","dfs.bytes-per-checksum":"512","mapreduce.job.acl-view-job":" ","fs.s3a.s3guard.ddb.background.sleep":"25ms","fs.s3a.retry.limit":"${fs.s3a.attempts.maximum}","mapreduce.jobhistory.loadedjobs.cache.size":"5","fs.s3a.s3guard.ddb.table.create":"false","yarn.nodemanager.amrmproxy.enabled":"false","yarn.timeline-service.entity-group-fs-store.with-user-dir":"false","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.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.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","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","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.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.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","yarn.timeline-service.client.fd-clean-interval-secs":"60","yarn.resourcemanager.nodemanagers.heartbeat-interval-ms":"1000","hadoop.common.configuration.version":"3.0.0","fs.s3a.s3guard.ddb.table.capacity.read":"500","yarn.nodemanager.remote-app-log-dir-suffix":"logs","yarn.nodemanager.windows-container.cpu-limit.enabled":"false","yarn.nodemanager.runtime.linux.docker.privileged-containers.allowed":"false","file.blocksize":"67108864","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","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","fs.har.impl.disable.cache":"true","yarn.webapp.ui2.enable":"false","io.compression.codec.bzip2.library":"system-native","yarn.nodemanager.distributed-scheduling.enabled":"false","mapreduce.shuffle.connection-keep-alive.timeout":"5","yarn.resourcemanager.webapp.https.address":"${yarn.resourcemanager.hostname}:8090","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.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","fs.ftp.transfer.mode":"BLOCK_TRANSFER_MODE","ipc.server.log.slow.rpc":"false","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","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","hadoop.security.credential.clear-text-fallback":"true","yarn.nodemanager.collector-service.thread-count":"5","fs.azure.secure.mode":"false","mapreduce.jobhistory.joblist.cache.size":"20000","fs.ftp.host":"0.0.0.0","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":"false","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","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","yarn.client.load.resource-types.from-server":"false","ha.zookeeper.session-timeout.ms":"10000","tfile.io.chunk.size":"1048576","fs.s3a.s3guard.ddb.table.capacity.write":"100","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","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","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","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","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","mapreduce.job.max.split.locations":"15","yarn.resourcemanager.nm-container-queuing.max-queue-length":"15","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.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","fs.s3a.buffer.dir":"${hadoop.tmp.dir}/s3a","hadoop.ssl.enabled.protocols":"TLSv1,SSLv2Hello,TLSv1.1,TLSv1.2","mapreduce.jobhistory.admin.address":"0.0.0.0:10033","yarn.log-aggregation-status.time-out.ms":"600000","fs.s3a.assumed.role.sts.endpoint.region":"us-west-1","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.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","fs.azure.local.sas.key.mode":"false","ipc.maximum.data.length":"67108864","mapreduce.shuffle.max.threads":"0","yarn.router.pipeline.cache-max-size":"25","yarn.resourcemanager.nm-container-queuing.load-comparator":"QUEUE_LENGTH","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","mapreduce.jobhistory.http.policy":"HTTP_ONLY","yarn.sharedcache.store.in-memory.check-period-mins":"720","hadoop.security.group.mapping.ldap.ssl":"false","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.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.nodemanager.resourcemanager.minimum.version":"NONE","mapreduce.job.speculative.speculative-cap-running-tasks":"0.1","yarn.admin.acl":"*","yarn.nodemanager.recovery.supervised":"false","yarn.sharedcache.admin.thread-count":"1","yarn.resourcemanager.ha.automatic-failover.enabled":"true","mapreduce.reduce.skip.maxgroups":"0","mapreduce.reduce.shuffle.connect.timeout":"180000","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","hadoop.ssl.enabled":"false","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":"10","mapreduce.reduce.shuffle.input.buffer.percent":"0.70","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.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":"2147483647","yarn.resourcemanager.webapp.xfs-filter.xframe-options":"SAMEORIGIN","mapreduce.jobhistory.done-dir":"${yarn.app.mapreduce.am.staging-dir}/history/done","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","yarn.webapp.filter-entity-list-by-user":"false","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}","hadoop.workaround.non.threadsafe.getpwuid":"true","fs.df.interval":"60000","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","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","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.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","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.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","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","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.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","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","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":"${fs.s3a.attempts.maximum}","hadoop.http.authentication.type":"simple","mapreduce.job.cache.limit.max-resources":"0","mapreduce.task.userlog.limit.kb":"0","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","hadoop.user.group.static.mapping.overrides":"dr.who=;","fs.azure.sas.expiry.period":"90d","mapreduce.jobhistory.recovery.store.class":"org.apache.hadoop.mapreduce.v2.hs.HistoryServerFileSystemStateStoreService","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","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","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":"1000ms","seq.io.sort.factor":"100","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/","mapreduce.task.profile":"false","yarn.resourcemanager.fs.state-store.uri":"${hadoop.tmp.dir}/yarn/system/rmstore","mapreduce.jobhistory.always-scan-user-dir":"false","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","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","fs.automatic.close":"true","yarn.nodemanager.hostname":"0.0.0.0","yarn.nodemanager.resource.memory.cgroups.swappiness":"0","ftp.stream-buffer-size":"4096","yarn.fail-fast":"false","yarn.timeline-service.app-aggregation-interval-secs":"15","hadoop.security.group.mapping.ldap.search.filter.user":"(&(objectClass=user)(sAMAccountName={0}))","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","yarn.resourcemanager.delegation.token.max-lifetime":"*********(redacted)","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.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.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.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","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","hadoop.security.dns.log-slow-lookups.enabled":"false","mapreduce.job.committer.setup.cleanup.needed":"true","mapreduce.job.running.reduce.limit":"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","fs.s3a.max.total.tasks":"5","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.nodemanager.health-checker.script.timeout-ms":"1200000","yarn.timeline-service.leveldb-timeline-store.start-time-write-cache-size":"10000","yarn.resourcemanager.resource-profiles.enabled":"false","yarn.timeline-service.hbase-schema.prefix":"prod.","fs.azure.authorization":"false","mapreduce.map.log.level":"INFO","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","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","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","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":"0,1","yarn.timeline-service.client.internal-timers-ttl-secs":"420","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","ipc.client.tcpnodelay":"true","ipc.client.rpc-timeout.ms":"0","yarn.nodemanager.webapp.rest-csrf.methods-to-ignore":"GET,OPTIONS,HEAD","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","fs.s3a.socket.recv.buffer":"8192","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.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","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","mapreduce.shuffle.ssl.enabled":"false","yarn.timeline-service.hbase.coprocessor.app-final-value-retention-milliseconds":"259200000","fs.s3a.committer.staging.abort.pending.uploads":"true","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","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":"100M","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","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","yarn.resourcemanager.ha.enabled":"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.federation.subcluster-resolver.class":"org.apache.hadoop.yarn.server.federation.resolver.DefaultSubClusterResolverImpl","yarn.resourcemanager.zk-state-store.parent-path":"/rmstore","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":"fail","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","fs.AbstractFileSystem.adl.impl":"org.apache.hadoop.fs.adl.Adl","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":"128","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.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.federation.registry.base-dir":"yarnfederation/","mapreduce.job.max.map":"-1","mapreduce.job.local-fs.single-disk-limit.bytes":"-1","mapreduce.job.ubertask.maxreduces":"1","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","yarn.client.nodemanager-connect.retry-interval-ms":"10000","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","yarn.timeline-service.webapp.rest-csrf.enabled":"false"},"System Properties":{"EMR_CLUSTER_ID":"j-123AB678XY321","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":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Private Build","java.vm.specification.version":"1.8","user.home":"/home/user1","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/amd64","user.dir":"/home/user1/workspace/spark-rapids-another","java.library.path":"/usr/local/cuda/lib64:/usr/java/packages/lib/amd64:/usr/lib/x86_64-linux-gnu/jni:/lib/x86_64-linux-gnu:/usr/lib/x86_64-linux-gnu:/usr/lib/jni:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.282-b08","jetty.git.hash":"e46af88704a893fc12cb0e3bf46e2c7b48a009e7","java.endorsed.dirs":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/endorsed","java.runtime.version":"1.8.0_282-8u282-b08-0ubuntu1~18.04-b08","java.vm.info":"mixed mode","java.ext.dirs":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"OpenJDK Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/resources.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/rt.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/sunrsasign.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jsse.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jce.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/charsets.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jfr.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"4.15.0-130-generic","sun.os.patch.level":"unknown","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","user.language":"en","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.print.PSPrinterJob","java.awt.graphicsenv":"sun.awt.X11GraphicsEnvironment","awt.toolkit":"sun.awt.X11.XToolkit","os.name":"Linux","java.vm.vendor":"Private Build","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"user1","java.vm.name":"OpenJDK 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master local[1] --conf spark.driver.memory=30g --class com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils --jars /home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar /home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar dsAndDf /home/user1/testeventlogDir 100001","java.home":"/usr/lib/jvm/java-8-openjdk-amd64/jre","java.version":"1.8.0_282","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-exec-2.3.7-core.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aopalliance-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/breeze-macros_2.12-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-yarn_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jaxb-api-2.2.11.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-scala_2.12-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okhttp-2.7.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-scheduling-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-jaxrs-json-provider-2.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-storageclass-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-sketch_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/flatbuffers-java-1.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-media-jaxb-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/chill_2.12-0.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-policy-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guava-14.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jline-2.14.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stax-api-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/chill-java-0.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/shims-0.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jdo-api-3.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-autoscaling-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/gson-2.2.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/py4j-0.10.9.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-core_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-memory-netty-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-jaxb-annotations-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-kubernetes_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-admin-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/slf4j-api-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stax2-api-3.1.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-core_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-xdr-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-common-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-apiextensions-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/antlr4-runtime-4.8-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-common-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/antlr-runtime-3.5.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-core-4.1.17.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mesos_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-streaming_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jsp-api-2.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-framework-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/activation-1.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/mesos-1.4.0-shaded-protobuf.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/pyrolite-4.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/libthrift-0.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-ipc-1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-scheduler-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.annotation-api-1.3.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-client-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-crypto-1.1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-util-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-events-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-databind-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/xbean-asm7-shaded-4.15.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-mapreduce-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/core-1.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okio-1.14.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-discovery-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zookeeper-3.4.14.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/woodstox-core-5.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.xml.bind-api-2.3.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-apps-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-service-rpc-3.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mllib-local_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-launcher_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/httpcore-4.4.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jcl-over-slf4j-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-tags_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-jdbc-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-client-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arpack_combined_all-0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/token-provider-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/joda-time-2.10.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-graphx_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-format-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/re2j-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kryo-shaded-4.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jpam-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-hadoop-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/super-csv-2.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/breeze_2.12-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-jackson_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-collections-3.2.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-core-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-beeline-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-ast_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-xml_2.12-1.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-server-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/bonecp-0.8.0.RELEASE.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/slf4j-log4j12-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-shims-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/leveldbjni-all-1.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-reflect-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-serde-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/audience-annotations-0.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-batch-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-coordination-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-common-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-paranamer-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/threeten-extra-1.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-client-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/compress-lzf-1.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javax.inject-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/conf/":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-core-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/accessors-smart-1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-math3-3.4.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/RoaringBitmap-0.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ehcache-3.3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-column-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.activation-api-1.2.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-configuration2-2.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-api-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-recipes-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-network-shuffle_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/xz-1.5.jar":"System Classpath","spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar":"Added By User","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/dropwizard-metrics-hadoop-metrics2-reporter-0.1.2.jar":"System Classpath","spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar":"Added By User","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aircompressor-0.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-graphite-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-jmx-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-server-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/protobuf-java-2.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-util_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-lang3-3.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-memory-core-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-vector-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/generex-1.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/httpclient-4.5.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-client-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-compress-1.20.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-metastore-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-auth-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-scalap_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.ws.rs-api-2.1.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ST4-4.0.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-locator-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-networking-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-extensions-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jaxb-runtime-2.3.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-compiler-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/shapeless_2.12-2.3.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-core-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-pool-1.5.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-metrics-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jta-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ivy-2.4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/paranamer-2.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/istack-commons-runtime-3.0.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-text-1.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-cli-1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-core-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-net-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-utils-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/transaction-api-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/automaton-1.11-8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-parser-combinators_2.12-1.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/cats-kernel_2.12-2.0.0-M4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/nimbus-jose-jwt-4.41.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-util-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-container-servlet-core-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-vector-code-gen-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.validation-api-2.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-jaxrs-base-2.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-common-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-rdbms-4.1.19.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/janino-3.0.16.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/algebra_2.12-2.0.0-M2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/velocity-1.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.servlet-api-4.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-jobclient-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/oro-2.0.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/htrace-core4-4.1.0-incubating.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-mapred-1.8.2-hadoop2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-datatype-jsr310-2.11.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-kvstore_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-repl_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jul-to-slf4j-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-codec-1.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-cli-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-hk2-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guice-4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javassist-3.25.0-GA.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-lang-2.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/macro-compat_2.12-1.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-io-2.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-hive-thriftserver_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-dbcp-1.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-core-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-0.23-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-encoding-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okhttp-3.12.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guice-servlet-4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-hdfs-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-daemon-1.0.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-library-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-api-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jsr305-3.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json-1.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-network-common_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zstd-jni-1.4.8-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/snappy-java-1.1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-macros_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-annotations-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-simplekdc-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-logging-1.1.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stream-2.9.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-httpclient-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-hive_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-jackson-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-format-2.4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-core-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json-smart-2.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javax.jdo-3.2.0-m3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zjsonpatch-0.3.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-pkix-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/opencsv-2.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/machinist_2.12-0.6.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aopalliance-repackaged-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-platform_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/netty-all-4.1.51.Final.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-admissionregistration-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-rbac-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/osgi-resource-locator-1.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-annotations-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-config-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/libfb303-0.9.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-unsafe_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-beanutils-1.9.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-tags_2.12-3.1.1-tests.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-llap-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mllib_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-identity-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/log4j-1.2.17.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-settings-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-collection-compat_2.12-2.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-jvm-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/derby-10.12.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-storage-api-2.7.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/JLargeArrays-1.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-api-jdo-4.2.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/JTransforms-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javolution-5.5.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/geronimo-jcache_1.0_spec-1.0-alpha-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/minlog-1.3.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-container-servlet-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/objenesis-2.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/logging-interceptor-3.12.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-compiler-3.0.16.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-certificates-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jodd-core-3.5.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-sql_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jcip-annotations-1.0-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-asn1-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-json-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.inject-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-server-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-dataformat-yaml-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-server-web-proxy-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-registry-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-catalyst_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/HikariCP-2.5.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-crypto-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/dnsjava-2.1.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/univocity-parsers-2.9.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/lz4-java-1.7.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/snakeyaml-1.24.jar":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Rapids Spark Profiling Tool Unit Tests","App ID":"local-1622043423017","Timestamp":1622043422343,"User":"user1"} {"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":0,"description":"json at QualificationInfoUtils.scala:76","details":"org.apache.spark.sql.DataFrameWriter.json(DataFrameWriter.scala:853)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1(QualificationInfoUtils.scala:76)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1$adapted(QualificationInfoUtils.scala:73)\norg.apache.spark.rapids.TestUtils$.withTempPath(TestUtils.scala:31)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genDatasetEventLog(QualificationInfoUtils.scala:73)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:185)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nExecute InsertIntoHadoopFsRelationCommand (8)\n+- * SerializeFromObject (7)\n +- * MapElements (6)\n +- * Filter (5)\n +- * DeserializeToObject (4)\n +- * Project (3)\n +- * SerializeFromObject (2)\n +- Scan (1)\n\n\n(1) Scan\nOutput [1]: [obj#3]\nArguments: obj#3: scala.Tuple3, ParallelCollectionRDD[0] at parallelize at QualificationInfoUtils.scala:64\n\n(2) SerializeFromObject [codegen id : 1]\nInput [1]: [obj#3]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._1, true, false) AS _1#4, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._2, true, false) AS _2#5, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._3 AS _3#6]\n\n(3) Project [codegen id : 1]\nOutput [3]: [_1#4 AS name#11, _2#5 AS friend#12, _3#6 AS age#13]\nInput [3]: [_1#4, _2#5, _3#6]\n\n(4) DeserializeToObject [codegen id : 1]\nInput [3]: [name#11, friend#12, age#13]\nArguments: newInstance(class com.nvidia.spark.rapids.tool.profiling.RapidsFriends), obj#30: com.nvidia.spark.rapids.tool.profiling.RapidsFriends\n\n(5) Filter [codegen id : 1]\nInput [1]: [obj#30]\nCondition : com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$$$Lambda$1503/443638755@31f31b20.apply\n\n(6) MapElements [codegen id : 1]\nInput [1]: [obj#30]\nArguments: com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$$$Lambda$1505/1108374296@6723cce7, obj#31: scala.Tuple2\n\n(7) SerializeFromObject [codegen id : 1]\nInput [1]: [obj#31]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1, true, false) AS _1#32, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._2 AS _2#33]\n\n(8) Execute InsertIntoHadoopFsRelationCommand\nInput [2]: [_1#32, _2#33]\nArguments: file:/tmp/spark-4a9e661e-fc37-4f5f-8bbc-1647dbaebbaf, false, JSON, Map(path -> /tmp/spark-4a9e661e-fc37-4f5f-8bbc-1647dbaebbaf), ErrorIfExists, [_1, _2]\n\n","sparkPlanInfo":{"nodeName":"Execute InsertIntoHadoopFsRelationCommand","simpleString":"Execute InsertIntoHadoopFsRelationCommand file:/tmp/spark-4a9e661e-fc37-4f5f-8bbc-1647dbaebbaf, false, JSON, Map(path -> /tmp/spark-4a9e661e-fc37-4f5f-8bbc-1647dbaebbaf), ErrorIfExists, [_1, _2]","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1, true, false) AS _1#32, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._2 AS _2#33]","children":[{"nodeName":"MapElements","simpleString":"MapElements com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$$$Lambda$1505/1108374296@6723cce7, obj#31: scala.Tuple2","children":[{"nodeName":"Filter","simpleString":"Filter com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$$$Lambda$1503/443638755@31f31b20.apply","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject newInstance(class com.nvidia.spark.rapids.tool.profiling.RapidsFriends), obj#30: com.nvidia.spark.rapids.tool.profiling.RapidsFriends","children":[{"nodeName":"Project","simpleString":"Project [_1#4 AS name#11, _2#5 AS friend#12, _3#6 AS age#13]","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._1, true, false) AS _1#4, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._2, true, false) AS _2#5, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._3 AS _3#6]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Scan","simpleString":"Scan[obj#3]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":6,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":5,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":4,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of written files","accumulatorId":0,"metricType":"sum"},{"name":"written output","accumulatorId":1,"metricType":"size"},{"name":"number of output rows","accumulatorId":2,"metricType":"sum"},{"name":"number of dynamic part","accumulatorId":3,"metricType":"sum"}]},"time":1622043425909} -{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1622043426367,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"json at QualificationInfoUtils.scala:76","Number of Tasks":12,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"Scan\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at QualificationInfoUtils.scala:64","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameWriter.json(DataFrameWriter.scala:853)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1(QualificationInfoUtils.scala:76)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1$adapted(QualificationInfoUtils.scala:73)\norg.apache.spark.rapids.TestUtils$.withTempPath(TestUtils.scala:31)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genDatasetEventLog(QualificationInfoUtils.scala:73)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:185)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[0],"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"1\",\"name\":\"Execute InsertIntoHadoopFsRelationCommand\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"0","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"json at QualificationInfoUtils.scala:76","Number of Tasks":12,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"Scan\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at QualificationInfoUtils.scala:64","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameWriter.json(DataFrameWriter.scala:853)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1(QualificationInfoUtils.scala:76)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1$adapted(QualificationInfoUtils.scala:73)\norg.apache.spark.rapids.TestUtils$.withTempPath(TestUtils.scala:31)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genDatasetEventLog(QualificationInfoUtils.scala:73)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:185)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043426387,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"1\",\"name\":\"Execute InsertIntoHadoopFsRelationCommand\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"0","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"}} +{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1622043426367,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"json at QualificationInfoUtils.scala:76","Number of Tasks":12,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"Scan\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at QualificationInfoUtils.scala:64","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameWriter.json(DataFrameWriter.scala:853)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1(QualificationInfoUtils.scala:76)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1$adapted(QualificationInfoUtils.scala:73)\norg.apache.spark.rapids.TestUtils$.withTempPath(TestUtils.scala:31)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genDatasetEventLog(QualificationInfoUtils.scala:73)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:185)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[0],"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"1\",\"name\":\"Execute InsertIntoHadoopFsRelationCommand\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"0","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423017"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"json at QualificationInfoUtils.scala:76","Number of Tasks":12,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"Scan\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at QualificationInfoUtils.scala:64","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameWriter.json(DataFrameWriter.scala:853)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1(QualificationInfoUtils.scala:76)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1$adapted(QualificationInfoUtils.scala:73)\norg.apache.spark.rapids.TestUtils$.withTempPath(TestUtils.scala:31)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genDatasetEventLog(QualificationInfoUtils.scala:73)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:185)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043426387,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"1\",\"name\":\"Execute InsertIntoHadoopFsRelationCommand\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"0","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423017"}} {"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1622043426546,"Executor ID":"driver","Host":"10.10.10.100","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} {"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1622043426567,"Executor ID":"driver","Host":"10.10.10.100","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} {"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1622043426568,"Executor ID":"driver","Host":"10.10.10.100","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} @@ -39,9 +39,9 @@ {"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":0,"accumUpdates":[[0,12],[1,16560],[2,720],[3,0]]} {"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":0,"time":1622043427215} {"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":1,"description":"count at QualificationInfoUtils.scala:94","details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\n* HashAggregate (15)\n+- Exchange (14)\n +- * HashAggregate (13)\n +- * Project (12)\n +- * SortMergeJoin Inner (11)\n :- * Sort (5)\n : +- Exchange (4)\n : +- * Project (3)\n : +- * SerializeFromObject (2)\n : +- Scan (1)\n +- * Sort (10)\n +- Exchange (9)\n +- * Project (8)\n +- * SerializeFromObject (7)\n +- Scan (6)\n\n\n(1) Scan\nOutput [1]: [obj#40]\nArguments: obj#40: int, ParallelCollectionRDD[5] at makeRDD at QualificationInfoUtils.scala:92\n\n(2) SerializeFromObject [codegen id : 1]\nInput [1]: [obj#40]\nArguments: [input[0, int, false] AS value#41]\n\n(3) Project [codegen id : 1]\nOutput [1]: [value#41 AS a#51]\nInput [1]: [value#41]\n\n(4) Exchange\nInput [1]: [a#51]\nArguments: hashpartitioning(a#51, 200), ENSURE_REQUIREMENTS, [id=#93]\n\n(5) Sort [codegen id : 2]\nInput [1]: [a#51]\nArguments: [a#51 ASC NULLS FIRST], false, 0\n\n(6) Scan\nOutput [1]: [obj#46]\nArguments: obj#46: int, ParallelCollectionRDD[6] at makeRDD at QualificationInfoUtils.scala:93\n\n(7) SerializeFromObject [codegen id : 3]\nInput [1]: [obj#46]\nArguments: [input[0, int, false] AS value#47]\n\n(8) Project [codegen id : 3]\nOutput [1]: [value#47 AS b#53]\nInput [1]: [value#47]\n\n(9) Exchange\nInput [1]: [b#53]\nArguments: hashpartitioning(b#53, 200), ENSURE_REQUIREMENTS, [id=#102]\n\n(10) Sort [codegen id : 4]\nInput [1]: [b#53]\nArguments: [b#53 ASC NULLS FIRST], false, 0\n\n(11) SortMergeJoin [codegen id : 5]\nLeft keys [1]: [a#51]\nRight keys [1]: [b#53]\nJoin condition: None\n\n(12) Project [codegen id : 5]\nOutput: []\nInput [2]: [a#51, b#53]\n\n(13) HashAggregate [codegen id : 5]\nInput: []\nKeys: []\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count#64L]\nResults [1]: [count#65L]\n\n(14) Exchange\nInput [1]: [count#65L]\nArguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#111]\n\n(15) HashAggregate [codegen id : 6]\nInput [1]: [count#65L]\nKeys: []\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#61L]\nResults [1]: [count(1)#61L AS count#62L]\n\n","sparkPlanInfo":{"nodeName":"WholeStageCodegen (6)","simpleString":"WholeStageCodegen (6)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Exchange","simpleString":"Exchange SinglePartition, ENSURE_REQUIREMENTS, [id=#111]","children":[{"nodeName":"WholeStageCodegen (5)","simpleString":"WholeStageCodegen (5)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[], functions=[partial_count(1)])","children":[{"nodeName":"Project","simpleString":"Project","children":[{"nodeName":"SortMergeJoin","simpleString":"SortMergeJoin [a#51], [b#53], Inner","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"Sort","simpleString":"Sort [a#51 ASC NULLS FIRST], false, 0","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(a#51, 200), ENSURE_REQUIREMENTS, [id=#93]","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [value#41 AS a#51]","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [input[0, int, false] AS value#41]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Scan","simpleString":"Scan[obj#40]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":116,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":115,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":74,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":75,"metricType":"nsTiming"},{"name":"records read","accumulatorId":72,"metricType":"sum"},{"name":"local bytes read","accumulatorId":70,"metricType":"size"},{"name":"fetch wait time","accumulatorId":71,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":68,"metricType":"size"},{"name":"local blocks read","accumulatorId":67,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":66,"metricType":"sum"},{"name":"data size","accumulatorId":65,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":69,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":73,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"sort time","accumulatorId":112,"metricType":"timing"},{"name":"peak memory","accumulatorId":113,"metricType":"size"},{"name":"spill size","accumulatorId":114,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":111,"metricType":"timing"}]}],"metadata":{},"metrics":[]},{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"Sort","simpleString":"Sort [b#53 ASC NULLS FIRST], false, 0","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(b#53, 200), ENSURE_REQUIREMENTS, [id=#102]","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"Project","simpleString":"Project [value#47 AS b#53]","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [input[0, int, false] AS value#47]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Scan","simpleString":"Scan[obj#46]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":122,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":121,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":85,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":86,"metricType":"nsTiming"},{"name":"records read","accumulatorId":83,"metricType":"sum"},{"name":"local bytes read","accumulatorId":81,"metricType":"size"},{"name":"fetch wait time","accumulatorId":82,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":79,"metricType":"size"},{"name":"local blocks read","accumulatorId":78,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":77,"metricType":"sum"},{"name":"data size","accumulatorId":76,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":80,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":84,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"sort time","accumulatorId":118,"metricType":"timing"},{"name":"peak memory","accumulatorId":119,"metricType":"size"},{"name":"spill size","accumulatorId":120,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":117,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":110,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":107,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":108,"metricType":"timing"},{"name":"peak memory","accumulatorId":106,"metricType":"size"},{"name":"number of output rows","accumulatorId":105,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":109,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":104,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":96,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":97,"metricType":"nsTiming"},{"name":"records read","accumulatorId":94,"metricType":"sum"},{"name":"local bytes read","accumulatorId":92,"metricType":"size"},{"name":"fetch wait time","accumulatorId":93,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":90,"metricType":"size"},{"name":"local blocks read","accumulatorId":89,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":88,"metricType":"sum"},{"name":"data size","accumulatorId":87,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":91,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":95,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":101,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":102,"metricType":"timing"},{"name":"peak memory","accumulatorId":100,"metricType":"size"},{"name":"number of output rows","accumulatorId":99,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":103,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":98,"metricType":"timing"}]},"time":1622043427414} -{"Event":"SparkListenerJobStart","Job ID":1,"Submission Time":1622043427637,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"43\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"15\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:93","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"39\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"14\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:92","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"29\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":200,"RDD Info":[{"RDD ID":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[18],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[14],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"20\",\"name\":\"WholeStageCodegen (5)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"35\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"25\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"20\",\"name\":\"WholeStageCodegen (5)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[11,16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[1,2],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":1,"RDD Info":[{"RDD ID":22,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"44\",\"name\":\"mapPartitionsInternal\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":21,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"16\",\"name\":\"WholeStageCodegen (6)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"19\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[3],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[1,2,3,4],"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"43\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"15\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:93","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"39\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043427644,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"14\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:92","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"29\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043427656,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"}} +{"Event":"SparkListenerJobStart","Job ID":1,"Submission Time":1622043427637,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"43\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"15\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:93","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"39\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"14\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:92","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"29\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":200,"RDD Info":[{"RDD ID":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[18],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[14],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"20\",\"name\":\"WholeStageCodegen (5)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"35\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"25\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"20\",\"name\":\"WholeStageCodegen (5)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[11,16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[1,2],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":1,"RDD Info":[{"RDD ID":22,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"44\",\"name\":\"mapPartitionsInternal\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":21,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"16\",\"name\":\"WholeStageCodegen (6)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"19\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[3],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[1,2,3,4],"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423017"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"43\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"15\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:93","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"39\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043427644,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423017"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"14\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:92","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"29\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043427656,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423017"}} {"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":0,"Attempt":0,"Launch Time":1622043427655,"Executor ID":"driver","Host":"10.10.10.100","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} {"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":1,"Attempt":0,"Launch Time":1622043427661,"Executor ID":"driver","Host":"10.10.10.100","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} {"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":2,"Attempt":0,"Launch Time":1622043427661,"Executor ID":"driver","Host":"10.10.10.100","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} diff --git a/core/src/test/resources/spark-events-qualification/cluster_information/platform/onprem b/core/src/test/resources/spark-events-qualification/cluster_information/platform/onprem index f8b95ddf3..75743d6ac 100644 --- a/core/src/test/resources/spark-events-qualification/cluster_information/platform/onprem +++ b/core/src/test/resources/spark-events-qualification/cluster_information/platform/onprem @@ -5,11 +5,11 @@ {"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"10.10.10.228","Port":46767},"Maximum Memory":11884245811,"Timestamp":1702417716594,"Maximum Onheap Memory":11884245811,"Maximum Offheap Memory":0} {"Event":"SparkListenerExecutorAdded","Timestamp":1702417716738,"Executor ID":"0","Executor Info":{"Host":"10.10.10.244","Total Cores":8,"Log Urls":{"stdout":"http://10.10.10.244:40000/logPage/?appId=app-20231212214826-0000&executorId=0&logType=stdout","stderr":"http://10.10.10.244:40000/logPage/?appId=app-20231212214826-0000&executorId=0&logType=stderr"},"Attributes":{},"Resources":{},"Resource Profile Id":0}} {"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"0","Host":"10.10.10.244","Port":43349},"Maximum Memory":11884245811,"Timestamp":1702417718124,"Maximum Onheap Memory":11884245811,"Maximum Offheap Memory":0} -{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/lib/jvm/java-8-openjdk-amd64/jre","Java Version":"1.8.0_282 (Private Build)","Scala Version":"version 2.12.10"},"Spark Properties":{"spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.scheduler.mode":"FIFO","spark.driver.memory":"30g","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"},"Hadoop Properties":{"hadoop.service.shutdown.timeout":"30s","yarn.resourcemanager.amlauncher.thread-count":"50","yarn.sharedcache.enabled":"false","fs.s3a.connection.maximum":"15","yarn.nodemanager.numa-awareness.numactl.cmd":"/usr/bin/numactl","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","mapreduce.framework.name":"local","yarn.sharedcache.uploader.server.thread-count":"50","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","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","hadoop.registry.secure":"false","hadoop.shell.safely.delete.limit.num.files":"100","dfs.bytes-per-checksum":"512","mapreduce.job.acl-view-job":" ","fs.s3a.s3guard.ddb.background.sleep":"25ms","fs.s3a.retry.limit":"${fs.s3a.attempts.maximum}","mapreduce.jobhistory.loadedjobs.cache.size":"5","fs.s3a.s3guard.ddb.table.create":"false","yarn.nodemanager.amrmproxy.enabled":"false","yarn.timeline-service.entity-group-fs-store.with-user-dir":"false","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.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.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","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","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.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.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","yarn.timeline-service.client.fd-clean-interval-secs":"60","yarn.resourcemanager.nodemanagers.heartbeat-interval-ms":"1000","hadoop.common.configuration.version":"3.0.0","fs.s3a.s3guard.ddb.table.capacity.read":"500","yarn.nodemanager.remote-app-log-dir-suffix":"logs","yarn.nodemanager.windows-container.cpu-limit.enabled":"false","yarn.nodemanager.runtime.linux.docker.privileged-containers.allowed":"false","file.blocksize":"67108864","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","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","fs.har.impl.disable.cache":"true","yarn.webapp.ui2.enable":"false","io.compression.codec.bzip2.library":"system-native","yarn.nodemanager.distributed-scheduling.enabled":"false","mapreduce.shuffle.connection-keep-alive.timeout":"5","yarn.resourcemanager.webapp.https.address":"${yarn.resourcemanager.hostname}:8090","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.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","fs.ftp.transfer.mode":"BLOCK_TRANSFER_MODE","ipc.server.log.slow.rpc":"false","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","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","hadoop.security.credential.clear-text-fallback":"true","yarn.nodemanager.collector-service.thread-count":"5","fs.azure.secure.mode":"false","mapreduce.jobhistory.joblist.cache.size":"20000","fs.ftp.host":"0.0.0.0","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":"false","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","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","yarn.client.load.resource-types.from-server":"false","ha.zookeeper.session-timeout.ms":"10000","tfile.io.chunk.size":"1048576","fs.s3a.s3guard.ddb.table.capacity.write":"100","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","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","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","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","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","mapreduce.job.max.split.locations":"15","yarn.resourcemanager.nm-container-queuing.max-queue-length":"15","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.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","fs.s3a.buffer.dir":"${hadoop.tmp.dir}/s3a","hadoop.ssl.enabled.protocols":"TLSv1,SSLv2Hello,TLSv1.1,TLSv1.2","mapreduce.jobhistory.admin.address":"0.0.0.0:10033","yarn.log-aggregation-status.time-out.ms":"600000","fs.s3a.assumed.role.sts.endpoint.region":"us-west-1","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.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","fs.azure.local.sas.key.mode":"false","ipc.maximum.data.length":"67108864","mapreduce.shuffle.max.threads":"0","yarn.router.pipeline.cache-max-size":"25","yarn.resourcemanager.nm-container-queuing.load-comparator":"QUEUE_LENGTH","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","mapreduce.jobhistory.http.policy":"HTTP_ONLY","yarn.sharedcache.store.in-memory.check-period-mins":"720","hadoop.security.group.mapping.ldap.ssl":"false","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.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.nodemanager.resourcemanager.minimum.version":"NONE","mapreduce.job.speculative.speculative-cap-running-tasks":"0.1","yarn.admin.acl":"*","yarn.nodemanager.recovery.supervised":"false","yarn.sharedcache.admin.thread-count":"1","yarn.resourcemanager.ha.automatic-failover.enabled":"true","mapreduce.reduce.skip.maxgroups":"0","mapreduce.reduce.shuffle.connect.timeout":"180000","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","hadoop.ssl.enabled":"false","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":"10","mapreduce.reduce.shuffle.input.buffer.percent":"0.70","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.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":"2147483647","yarn.resourcemanager.webapp.xfs-filter.xframe-options":"SAMEORIGIN","mapreduce.jobhistory.done-dir":"${yarn.app.mapreduce.am.staging-dir}/history/done","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","yarn.webapp.filter-entity-list-by-user":"false","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}","hadoop.workaround.non.threadsafe.getpwuid":"true","fs.df.interval":"60000","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","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","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.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","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.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","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","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.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","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","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":"${fs.s3a.attempts.maximum}","hadoop.http.authentication.type":"simple","mapreduce.job.cache.limit.max-resources":"0","mapreduce.task.userlog.limit.kb":"0","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","hadoop.user.group.static.mapping.overrides":"dr.who=;","fs.azure.sas.expiry.period":"90d","mapreduce.jobhistory.recovery.store.class":"org.apache.hadoop.mapreduce.v2.hs.HistoryServerFileSystemStateStoreService","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","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","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":"1000ms","seq.io.sort.factor":"100","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/","mapreduce.task.profile":"false","yarn.resourcemanager.fs.state-store.uri":"${hadoop.tmp.dir}/yarn/system/rmstore","mapreduce.jobhistory.always-scan-user-dir":"false","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","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","fs.automatic.close":"true","yarn.nodemanager.hostname":"0.0.0.0","yarn.nodemanager.resource.memory.cgroups.swappiness":"0","ftp.stream-buffer-size":"4096","yarn.fail-fast":"false","yarn.timeline-service.app-aggregation-interval-secs":"15","hadoop.security.group.mapping.ldap.search.filter.user":"(&(objectClass=user)(sAMAccountName={0}))","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","yarn.resourcemanager.delegation.token.max-lifetime":"*********(redacted)","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.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.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.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","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","hadoop.security.dns.log-slow-lookups.enabled":"false","mapreduce.job.committer.setup.cleanup.needed":"true","mapreduce.job.running.reduce.limit":"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","fs.s3a.max.total.tasks":"5","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.nodemanager.health-checker.script.timeout-ms":"1200000","yarn.timeline-service.leveldb-timeline-store.start-time-write-cache-size":"10000","yarn.resourcemanager.resource-profiles.enabled":"false","yarn.timeline-service.hbase-schema.prefix":"prod.","fs.azure.authorization":"false","mapreduce.map.log.level":"INFO","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","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","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","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":"0,1","yarn.timeline-service.client.internal-timers-ttl-secs":"420","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","ipc.client.tcpnodelay":"true","ipc.client.rpc-timeout.ms":"0","yarn.nodemanager.webapp.rest-csrf.methods-to-ignore":"GET,OPTIONS,HEAD","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","fs.s3a.socket.recv.buffer":"8192","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.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","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","mapreduce.shuffle.ssl.enabled":"false","yarn.timeline-service.hbase.coprocessor.app-final-value-retention-milliseconds":"259200000","fs.s3a.committer.staging.abort.pending.uploads":"true","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","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":"100M","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","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","yarn.resourcemanager.ha.enabled":"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.federation.subcluster-resolver.class":"org.apache.hadoop.yarn.server.federation.resolver.DefaultSubClusterResolverImpl","yarn.resourcemanager.zk-state-store.parent-path":"/rmstore","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":"fail","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","fs.AbstractFileSystem.adl.impl":"org.apache.hadoop.fs.adl.Adl","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":"128","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.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.federation.registry.base-dir":"yarnfederation/","mapreduce.job.max.map":"-1","mapreduce.job.local-fs.single-disk-limit.bytes":"-1","mapreduce.job.ubertask.maxreduces":"1","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","yarn.client.nodemanager-connect.retry-interval-ms":"10000","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","yarn.timeline-service.webapp.rest-csrf.enabled":"false"},"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":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Private Build","java.vm.specification.version":"1.8","user.home":"/home/user1","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/amd64","user.dir":"/home/user1/workspace/spark-rapids-another","java.library.path":"/usr/local/cuda/lib64:/usr/java/packages/lib/amd64:/usr/lib/x86_64-linux-gnu/jni:/lib/x86_64-linux-gnu:/usr/lib/x86_64-linux-gnu:/usr/lib/jni:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.282-b08","jetty.git.hash":"e46af88704a893fc12cb0e3bf46e2c7b48a009e7","java.endorsed.dirs":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/endorsed","java.runtime.version":"1.8.0_282-8u282-b08-0ubuntu1~18.04-b08","java.vm.info":"mixed mode","java.ext.dirs":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"OpenJDK Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/resources.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/rt.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/sunrsasign.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jsse.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jce.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/charsets.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jfr.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"4.15.0-130-generic","sun.os.patch.level":"unknown","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","user.language":"en","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.print.PSPrinterJob","java.awt.graphicsenv":"sun.awt.X11GraphicsEnvironment","awt.toolkit":"sun.awt.X11.XToolkit","os.name":"Linux","java.vm.vendor":"Private Build","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"user1","java.vm.name":"OpenJDK 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master local[1] --conf spark.driver.memory=30g --class com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils --jars /home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar /home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar dsAndDf /home/user1/testeventlogDir 100001","java.home":"/usr/lib/jvm/java-8-openjdk-amd64/jre","java.version":"1.8.0_282","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-exec-2.3.7-core.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aopalliance-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/breeze-macros_2.12-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-yarn_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jaxb-api-2.2.11.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-scala_2.12-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okhttp-2.7.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-scheduling-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-jaxrs-json-provider-2.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-storageclass-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-sketch_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/flatbuffers-java-1.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-media-jaxb-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/chill_2.12-0.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-policy-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guava-14.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jline-2.14.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stax-api-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/chill-java-0.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/shims-0.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jdo-api-3.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-autoscaling-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/gson-2.2.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/py4j-0.10.9.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-core_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-memory-netty-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-jaxb-annotations-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-kubernetes_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-admin-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/slf4j-api-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stax2-api-3.1.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-core_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-xdr-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-common-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-apiextensions-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/antlr4-runtime-4.8-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-common-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/antlr-runtime-3.5.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-core-4.1.17.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mesos_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-streaming_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jsp-api-2.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-framework-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/activation-1.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/mesos-1.4.0-shaded-protobuf.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/pyrolite-4.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/libthrift-0.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-ipc-1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-scheduler-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.annotation-api-1.3.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-client-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-crypto-1.1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-util-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-events-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-databind-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/xbean-asm7-shaded-4.15.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-mapreduce-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/core-1.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okio-1.14.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-discovery-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zookeeper-3.4.14.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/woodstox-core-5.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.xml.bind-api-2.3.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-apps-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-service-rpc-3.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mllib-local_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-launcher_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/httpcore-4.4.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jcl-over-slf4j-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-tags_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-jdbc-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-client-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arpack_combined_all-0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/token-provider-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/joda-time-2.10.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-graphx_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-format-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/re2j-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kryo-shaded-4.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jpam-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-hadoop-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/super-csv-2.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/breeze_2.12-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-jackson_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-collections-3.2.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-core-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-beeline-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-ast_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-xml_2.12-1.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-server-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/bonecp-0.8.0.RELEASE.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/slf4j-log4j12-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-shims-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/leveldbjni-all-1.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-reflect-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-serde-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/audience-annotations-0.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-batch-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-coordination-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-common-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-paranamer-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/threeten-extra-1.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-client-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/compress-lzf-1.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javax.inject-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/conf/":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-core-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/accessors-smart-1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-math3-3.4.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/RoaringBitmap-0.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ehcache-3.3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-column-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.activation-api-1.2.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-configuration2-2.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-api-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-recipes-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-network-shuffle_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/xz-1.5.jar":"System Classpath","spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar":"Added By User","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/dropwizard-metrics-hadoop-metrics2-reporter-0.1.2.jar":"System Classpath","spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar":"Added By User","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aircompressor-0.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-graphite-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-jmx-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-server-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/protobuf-java-2.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-util_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-lang3-3.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-memory-core-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-vector-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/generex-1.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/httpclient-4.5.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-client-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-compress-1.20.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-metastore-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-auth-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-scalap_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.ws.rs-api-2.1.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ST4-4.0.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-locator-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-networking-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-extensions-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jaxb-runtime-2.3.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-compiler-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/shapeless_2.12-2.3.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-core-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-pool-1.5.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-metrics-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jta-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ivy-2.4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/paranamer-2.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/istack-commons-runtime-3.0.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-text-1.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-cli-1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-core-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-net-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-utils-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/transaction-api-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/automaton-1.11-8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-parser-combinators_2.12-1.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/cats-kernel_2.12-2.0.0-M4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/nimbus-jose-jwt-4.41.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-util-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-container-servlet-core-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-vector-code-gen-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.validation-api-2.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-jaxrs-base-2.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-common-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-rdbms-4.1.19.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/janino-3.0.16.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/algebra_2.12-2.0.0-M2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/velocity-1.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.servlet-api-4.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-jobclient-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/oro-2.0.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/htrace-core4-4.1.0-incubating.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-mapred-1.8.2-hadoop2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-datatype-jsr310-2.11.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-kvstore_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-repl_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jul-to-slf4j-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-codec-1.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-cli-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-hk2-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guice-4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javassist-3.25.0-GA.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-lang-2.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/macro-compat_2.12-1.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-io-2.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-hive-thriftserver_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-dbcp-1.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-core-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-0.23-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-encoding-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okhttp-3.12.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guice-servlet-4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-hdfs-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-daemon-1.0.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-library-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-api-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jsr305-3.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json-1.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-network-common_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zstd-jni-1.4.8-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/snappy-java-1.1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-macros_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-annotations-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-simplekdc-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-logging-1.1.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stream-2.9.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-httpclient-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-hive_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-jackson-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-format-2.4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-core-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json-smart-2.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javax.jdo-3.2.0-m3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zjsonpatch-0.3.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-pkix-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/opencsv-2.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/machinist_2.12-0.6.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aopalliance-repackaged-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-platform_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/netty-all-4.1.51.Final.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-admissionregistration-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-rbac-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/osgi-resource-locator-1.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-annotations-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-config-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/libfb303-0.9.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-unsafe_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-beanutils-1.9.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-tags_2.12-3.1.1-tests.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-llap-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mllib_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-identity-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/log4j-1.2.17.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-settings-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-collection-compat_2.12-2.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-jvm-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/derby-10.12.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-storage-api-2.7.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/JLargeArrays-1.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-api-jdo-4.2.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/JTransforms-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javolution-5.5.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/geronimo-jcache_1.0_spec-1.0-alpha-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/minlog-1.3.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-container-servlet-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/objenesis-2.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/logging-interceptor-3.12.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-compiler-3.0.16.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-certificates-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jodd-core-3.5.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-sql_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jcip-annotations-1.0-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-asn1-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-json-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.inject-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-server-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-dataformat-yaml-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-server-web-proxy-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-registry-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-catalyst_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/HikariCP-2.5.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-crypto-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/dnsjava-2.1.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/univocity-parsers-2.9.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/lz4-java-1.7.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/snakeyaml-1.24.jar":"System Classpath"}} -{"Event":"SparkListenerApplicationStart","App Name":"Rapids Spark Profiling Tool Unit Tests","App ID":"local-1622043423018","Timestamp":1622043422343,"User":"user1"} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/lib/jvm/java-8-openjdk-amd64/jre","Java Version":"1.8.0_282 (Private Build)","Scala Version":"version 2.12.10"},"Spark Properties":{"spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.scheduler.mode":"FIFO","spark.driver.memory":"30g","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423013"},"Hadoop Properties":{"hadoop.service.shutdown.timeout":"30s","yarn.resourcemanager.amlauncher.thread-count":"50","yarn.sharedcache.enabled":"false","fs.s3a.connection.maximum":"15","yarn.nodemanager.numa-awareness.numactl.cmd":"/usr/bin/numactl","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","mapreduce.framework.name":"local","yarn.sharedcache.uploader.server.thread-count":"50","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","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","hadoop.registry.secure":"false","hadoop.shell.safely.delete.limit.num.files":"100","dfs.bytes-per-checksum":"512","mapreduce.job.acl-view-job":" ","fs.s3a.s3guard.ddb.background.sleep":"25ms","fs.s3a.retry.limit":"${fs.s3a.attempts.maximum}","mapreduce.jobhistory.loadedjobs.cache.size":"5","fs.s3a.s3guard.ddb.table.create":"false","yarn.nodemanager.amrmproxy.enabled":"false","yarn.timeline-service.entity-group-fs-store.with-user-dir":"false","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.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.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","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","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.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.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","yarn.timeline-service.client.fd-clean-interval-secs":"60","yarn.resourcemanager.nodemanagers.heartbeat-interval-ms":"1000","hadoop.common.configuration.version":"3.0.0","fs.s3a.s3guard.ddb.table.capacity.read":"500","yarn.nodemanager.remote-app-log-dir-suffix":"logs","yarn.nodemanager.windows-container.cpu-limit.enabled":"false","yarn.nodemanager.runtime.linux.docker.privileged-containers.allowed":"false","file.blocksize":"67108864","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","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","fs.har.impl.disable.cache":"true","yarn.webapp.ui2.enable":"false","io.compression.codec.bzip2.library":"system-native","yarn.nodemanager.distributed-scheduling.enabled":"false","mapreduce.shuffle.connection-keep-alive.timeout":"5","yarn.resourcemanager.webapp.https.address":"${yarn.resourcemanager.hostname}:8090","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.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","fs.ftp.transfer.mode":"BLOCK_TRANSFER_MODE","ipc.server.log.slow.rpc":"false","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","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","hadoop.security.credential.clear-text-fallback":"true","yarn.nodemanager.collector-service.thread-count":"5","fs.azure.secure.mode":"false","mapreduce.jobhistory.joblist.cache.size":"20000","fs.ftp.host":"0.0.0.0","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":"false","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","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","yarn.client.load.resource-types.from-server":"false","ha.zookeeper.session-timeout.ms":"10000","tfile.io.chunk.size":"1048576","fs.s3a.s3guard.ddb.table.capacity.write":"100","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","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","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","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","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","mapreduce.job.max.split.locations":"15","yarn.resourcemanager.nm-container-queuing.max-queue-length":"15","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.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","fs.s3a.buffer.dir":"${hadoop.tmp.dir}/s3a","hadoop.ssl.enabled.protocols":"TLSv1,SSLv2Hello,TLSv1.1,TLSv1.2","mapreduce.jobhistory.admin.address":"0.0.0.0:10033","yarn.log-aggregation-status.time-out.ms":"600000","fs.s3a.assumed.role.sts.endpoint.region":"us-west-1","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.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","fs.azure.local.sas.key.mode":"false","ipc.maximum.data.length":"67108864","mapreduce.shuffle.max.threads":"0","yarn.router.pipeline.cache-max-size":"25","yarn.resourcemanager.nm-container-queuing.load-comparator":"QUEUE_LENGTH","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","mapreduce.jobhistory.http.policy":"HTTP_ONLY","yarn.sharedcache.store.in-memory.check-period-mins":"720","hadoop.security.group.mapping.ldap.ssl":"false","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.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.nodemanager.resourcemanager.minimum.version":"NONE","mapreduce.job.speculative.speculative-cap-running-tasks":"0.1","yarn.admin.acl":"*","yarn.nodemanager.recovery.supervised":"false","yarn.sharedcache.admin.thread-count":"1","yarn.resourcemanager.ha.automatic-failover.enabled":"true","mapreduce.reduce.skip.maxgroups":"0","mapreduce.reduce.shuffle.connect.timeout":"180000","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","hadoop.ssl.enabled":"false","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":"10","mapreduce.reduce.shuffle.input.buffer.percent":"0.70","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.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":"2147483647","yarn.resourcemanager.webapp.xfs-filter.xframe-options":"SAMEORIGIN","mapreduce.jobhistory.done-dir":"${yarn.app.mapreduce.am.staging-dir}/history/done","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","yarn.webapp.filter-entity-list-by-user":"false","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}","hadoop.workaround.non.threadsafe.getpwuid":"true","fs.df.interval":"60000","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","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","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.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","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.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","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","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.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","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","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":"${fs.s3a.attempts.maximum}","hadoop.http.authentication.type":"simple","mapreduce.job.cache.limit.max-resources":"0","mapreduce.task.userlog.limit.kb":"0","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","hadoop.user.group.static.mapping.overrides":"dr.who=;","fs.azure.sas.expiry.period":"90d","mapreduce.jobhistory.recovery.store.class":"org.apache.hadoop.mapreduce.v2.hs.HistoryServerFileSystemStateStoreService","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","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","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":"1000ms","seq.io.sort.factor":"100","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/","mapreduce.task.profile":"false","yarn.resourcemanager.fs.state-store.uri":"${hadoop.tmp.dir}/yarn/system/rmstore","mapreduce.jobhistory.always-scan-user-dir":"false","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","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","fs.automatic.close":"true","yarn.nodemanager.hostname":"0.0.0.0","yarn.nodemanager.resource.memory.cgroups.swappiness":"0","ftp.stream-buffer-size":"4096","yarn.fail-fast":"false","yarn.timeline-service.app-aggregation-interval-secs":"15","hadoop.security.group.mapping.ldap.search.filter.user":"(&(objectClass=user)(sAMAccountName={0}))","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","yarn.resourcemanager.delegation.token.max-lifetime":"*********(redacted)","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.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.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.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","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","hadoop.security.dns.log-slow-lookups.enabled":"false","mapreduce.job.committer.setup.cleanup.needed":"true","mapreduce.job.running.reduce.limit":"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","fs.s3a.max.total.tasks":"5","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.nodemanager.health-checker.script.timeout-ms":"1200000","yarn.timeline-service.leveldb-timeline-store.start-time-write-cache-size":"10000","yarn.resourcemanager.resource-profiles.enabled":"false","yarn.timeline-service.hbase-schema.prefix":"prod.","fs.azure.authorization":"false","mapreduce.map.log.level":"INFO","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","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","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","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":"0,1","yarn.timeline-service.client.internal-timers-ttl-secs":"420","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","ipc.client.tcpnodelay":"true","ipc.client.rpc-timeout.ms":"0","yarn.nodemanager.webapp.rest-csrf.methods-to-ignore":"GET,OPTIONS,HEAD","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","fs.s3a.socket.recv.buffer":"8192","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.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","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","mapreduce.shuffle.ssl.enabled":"false","yarn.timeline-service.hbase.coprocessor.app-final-value-retention-milliseconds":"259200000","fs.s3a.committer.staging.abort.pending.uploads":"true","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","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":"100M","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","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","yarn.resourcemanager.ha.enabled":"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.federation.subcluster-resolver.class":"org.apache.hadoop.yarn.server.federation.resolver.DefaultSubClusterResolverImpl","yarn.resourcemanager.zk-state-store.parent-path":"/rmstore","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":"fail","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","fs.AbstractFileSystem.adl.impl":"org.apache.hadoop.fs.adl.Adl","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":"128","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.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.federation.registry.base-dir":"yarnfederation/","mapreduce.job.max.map":"-1","mapreduce.job.local-fs.single-disk-limit.bytes":"-1","mapreduce.job.ubertask.maxreduces":"1","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","yarn.client.nodemanager-connect.retry-interval-ms":"10000","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","yarn.timeline-service.webapp.rest-csrf.enabled":"false"},"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":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Private Build","java.vm.specification.version":"1.8","user.home":"/home/user1","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/amd64","user.dir":"/home/user1/workspace/spark-rapids-another","java.library.path":"/usr/local/cuda/lib64:/usr/java/packages/lib/amd64:/usr/lib/x86_64-linux-gnu/jni:/lib/x86_64-linux-gnu:/usr/lib/x86_64-linux-gnu:/usr/lib/jni:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.282-b08","jetty.git.hash":"e46af88704a893fc12cb0e3bf46e2c7b48a009e7","java.endorsed.dirs":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/endorsed","java.runtime.version":"1.8.0_282-8u282-b08-0ubuntu1~18.04-b08","java.vm.info":"mixed mode","java.ext.dirs":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"OpenJDK Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/resources.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/rt.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/sunrsasign.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jsse.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jce.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/charsets.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jfr.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"4.15.0-130-generic","sun.os.patch.level":"unknown","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","user.language":"en","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.print.PSPrinterJob","java.awt.graphicsenv":"sun.awt.X11GraphicsEnvironment","awt.toolkit":"sun.awt.X11.XToolkit","os.name":"Linux","java.vm.vendor":"Private Build","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"user1","java.vm.name":"OpenJDK 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master local[1] --conf spark.driver.memory=30g --class com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils --jars /home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar /home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar dsAndDf /home/user1/testeventlogDir 100001","java.home":"/usr/lib/jvm/java-8-openjdk-amd64/jre","java.version":"1.8.0_282","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-exec-2.3.7-core.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aopalliance-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/breeze-macros_2.12-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-yarn_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jaxb-api-2.2.11.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-scala_2.12-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okhttp-2.7.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-scheduling-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-jaxrs-json-provider-2.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-storageclass-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-sketch_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/flatbuffers-java-1.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-media-jaxb-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/chill_2.12-0.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-policy-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guava-14.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jline-2.14.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stax-api-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/chill-java-0.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/shims-0.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jdo-api-3.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-autoscaling-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/gson-2.2.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/py4j-0.10.9.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-core_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-memory-netty-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-jaxb-annotations-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-kubernetes_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-admin-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/slf4j-api-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stax2-api-3.1.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-core_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-xdr-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-common-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-apiextensions-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/antlr4-runtime-4.8-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-common-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/antlr-runtime-3.5.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-core-4.1.17.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mesos_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-streaming_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jsp-api-2.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-framework-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/activation-1.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/mesos-1.4.0-shaded-protobuf.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/pyrolite-4.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/libthrift-0.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-ipc-1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-scheduler-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.annotation-api-1.3.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-client-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-crypto-1.1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-util-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-events-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-databind-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/xbean-asm7-shaded-4.15.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-mapreduce-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/core-1.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okio-1.14.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-discovery-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zookeeper-3.4.14.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/woodstox-core-5.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.xml.bind-api-2.3.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-apps-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-service-rpc-3.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mllib-local_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-launcher_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/httpcore-4.4.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jcl-over-slf4j-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-tags_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-jdbc-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-client-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arpack_combined_all-0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/token-provider-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/joda-time-2.10.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-graphx_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-format-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/re2j-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kryo-shaded-4.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jpam-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-hadoop-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/super-csv-2.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/breeze_2.12-1.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-jackson_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-collections-3.2.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-core-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-beeline-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-ast_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-xml_2.12-1.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-server-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/bonecp-0.8.0.RELEASE.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/slf4j-log4j12-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-shims-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/leveldbjni-all-1.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-reflect-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-serde-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/audience-annotations-0.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-batch-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-coordination-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-common-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-module-paranamer-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/threeten-extra-1.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-client-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/compress-lzf-1.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javax.inject-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/conf/":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/orc-core-1.5.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/accessors-smart-1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-math3-3.4.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/RoaringBitmap-0.9.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ehcache-3.3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-column-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.activation-api-1.2.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-configuration2-2.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-api-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/curator-recipes-2.13.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-network-shuffle_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/xz-1.5.jar":"System Classpath","spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar":"Added By User","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/dropwizard-metrics-hadoop-metrics2-reporter-0.1.2.jar":"System Classpath","spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar":"Added By User","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aircompressor-0.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-graphite-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-jmx-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-server-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/protobuf-java-2.5.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-util_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-lang3-3.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-memory-core-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/arrow-vector-2.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/generex-1.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/httpclient-4.5.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-client-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-compress-1.20.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-metastore-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-auth-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json4s-scalap_2.12-3.7.0-M5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.ws.rs-api-2.1.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ST4-4.0.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-locator-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-networking-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-extensions-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jaxb-runtime-2.3.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-compiler-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/shapeless_2.12-2.3.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-core-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-pool-1.5.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-metrics-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jta-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/ivy-2.4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/paranamer-2.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/istack-commons-runtime-3.0.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-text-1.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-cli-1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-core-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-net-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hk2-utils-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/transaction-api-1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/automaton-1.11-8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-parser-combinators_2.12-1.1.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/cats-kernel_2.12-2.0.0-M4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/nimbus-jose-jwt-4.41.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-util-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-container-servlet-core-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-vector-code-gen-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.validation-api-2.0.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-jaxrs-base-2.9.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-common-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-rdbms-4.1.19.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/janino-3.0.16.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/algebra_2.12-2.0.0-M2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/velocity-1.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.servlet-api-4.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-jobclient-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/oro-2.0.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/htrace-core4-4.1.0-incubating.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/avro-mapred-1.8.2-hadoop2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-datatype-jsr310-2.11.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-kvstore_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-repl_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jul-to-slf4j-1.7.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-codec-1.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-cli-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-hk2-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guice-4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javassist-3.25.0-GA.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-lang-2.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/macro-compat_2.12-1.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-io-2.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-hive-thriftserver_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-dbcp-1.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-core-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-shims-0.23-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-encoding-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/okhttp-3.12.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/guice-servlet-4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-hdfs-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-daemon-1.0.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-library-2.12.10.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-api-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jsr305-3.0.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json-1.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-network-common_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zstd-jni-1.4.8-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/snappy-java-1.1.8.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-macros_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-annotations-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-simplekdc-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-logging-1.1.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/stream-2.9.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-httpclient-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-hive_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-jackson-1.10.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/parquet-format-2.4.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-mapreduce-client-core-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/json-smart-2.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javax.jdo-3.2.0-m3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/zjsonpatch-0.3.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-pkix-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/opencsv-2.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/machinist_2.12-0.6.8.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/aopalliance-repackaged-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spire-platform_2.12-0.17.0-M1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/netty-all-4.1.51.Final.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-admissionregistration-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-rbac-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/osgi-resource-locator-1.0.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-annotations-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-config-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/libfb303-0.9.3.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-unsafe_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-beanutils-1.9.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-tags_2.12-3.1.1-tests.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-client-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-llap-common-2.3.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-mllib_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-identity-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/log4j-1.2.17.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-settings-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/scala-collection-compat_2.12-2.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-jvm-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/derby-10.12.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-storage-api-2.7.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/JLargeArrays-1.5.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/datanucleus-api-jdo-4.2.4.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/JTransforms-3.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/javolution-5.5.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/geronimo-jcache_1.0_spec-1.0-alpha-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/minlog-1.3.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-container-servlet-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/objenesis-2.6.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/logging-interceptor-3.12.12.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/commons-compiler-3.0.16.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kubernetes-model-certificates-4.12.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-common-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jodd-core-3.5.2.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-sql_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jcip-annotations-1.0-1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerby-asn1-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/metrics-json-4.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jakarta.inject-2.6.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jersey-server-2.30.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/jackson-dataformat-yaml-2.10.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-server-web-proxy-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hadoop-yarn-registry-3.2.0.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/spark-catalyst_2.12-3.1.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/HikariCP-2.5.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/kerb-crypto-1.0.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/dnsjava-2.1.7.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/univocity-parsers-2.9.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/lz4-java-1.7.1.jar":"System Classpath","/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/snakeyaml-1.24.jar":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Rapids Spark Profiling Tool Unit Tests","App ID":"local-1622043423013","Timestamp":1622043422343,"User":"user1"} {"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":0,"description":"json at QualificationInfoUtils.scala:76","details":"org.apache.spark.sql.DataFrameWriter.json(DataFrameWriter.scala:853)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1(QualificationInfoUtils.scala:76)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1$adapted(QualificationInfoUtils.scala:73)\norg.apache.spark.rapids.TestUtils$.withTempPath(TestUtils.scala:31)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genDatasetEventLog(QualificationInfoUtils.scala:73)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:185)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nExecute InsertIntoHadoopFsRelationCommand (8)\n+- * SerializeFromObject (7)\n +- * MapElements (6)\n +- * Filter (5)\n +- * DeserializeToObject (4)\n +- * Project (3)\n +- * SerializeFromObject (2)\n +- Scan (1)\n\n\n(1) Scan\nOutput [1]: [obj#3]\nArguments: obj#3: scala.Tuple3, ParallelCollectionRDD[0] at parallelize at QualificationInfoUtils.scala:64\n\n(2) SerializeFromObject [codegen id : 1]\nInput [1]: [obj#3]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._1, true, false) AS _1#4, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._2, true, false) AS _2#5, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._3 AS _3#6]\n\n(3) Project [codegen id : 1]\nOutput [3]: [_1#4 AS name#11, _2#5 AS friend#12, _3#6 AS age#13]\nInput [3]: [_1#4, _2#5, _3#6]\n\n(4) DeserializeToObject [codegen id : 1]\nInput [3]: [name#11, friend#12, age#13]\nArguments: newInstance(class com.nvidia.spark.rapids.tool.profiling.RapidsFriends), obj#30: com.nvidia.spark.rapids.tool.profiling.RapidsFriends\n\n(5) Filter [codegen id : 1]\nInput [1]: [obj#30]\nCondition : com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$$$Lambda$1503/443638755@31f31b20.apply\n\n(6) MapElements [codegen id : 1]\nInput [1]: [obj#30]\nArguments: com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$$$Lambda$1505/1108374296@6723cce7, obj#31: scala.Tuple2\n\n(7) SerializeFromObject [codegen id : 1]\nInput [1]: [obj#31]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1, true, false) AS _1#32, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._2 AS _2#33]\n\n(8) Execute InsertIntoHadoopFsRelationCommand\nInput [2]: [_1#32, _2#33]\nArguments: file:/tmp/spark-4a9e661e-fc37-4f5f-8bbc-1647dbaebbaf, false, JSON, Map(path -> /tmp/spark-4a9e661e-fc37-4f5f-8bbc-1647dbaebbaf), ErrorIfExists, [_1, _2]\n\n","sparkPlanInfo":{"nodeName":"Execute InsertIntoHadoopFsRelationCommand","simpleString":"Execute InsertIntoHadoopFsRelationCommand file:/tmp/spark-4a9e661e-fc37-4f5f-8bbc-1647dbaebbaf, false, JSON, Map(path -> /tmp/spark-4a9e661e-fc37-4f5f-8bbc-1647dbaebbaf), ErrorIfExists, [_1, _2]","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1, true, false) AS _1#32, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._2 AS _2#33]","children":[{"nodeName":"MapElements","simpleString":"MapElements com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$$$Lambda$1505/1108374296@6723cce7, obj#31: scala.Tuple2","children":[{"nodeName":"Filter","simpleString":"Filter com.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$$$Lambda$1503/443638755@31f31b20.apply","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject newInstance(class com.nvidia.spark.rapids.tool.profiling.RapidsFriends), obj#30: com.nvidia.spark.rapids.tool.profiling.RapidsFriends","children":[{"nodeName":"Project","simpleString":"Project [_1#4 AS name#11, _2#5 AS friend#12, _3#6 AS age#13]","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._1, true, false) AS _1#4, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._2, true, false) AS _2#5, knownnotnull(assertnotnull(input[0, scala.Tuple3, true]))._3 AS _3#6]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Scan","simpleString":"Scan[obj#3]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":6,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":5,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":4,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of written files","accumulatorId":0,"metricType":"sum"},{"name":"written output","accumulatorId":1,"metricType":"size"},{"name":"number of output rows","accumulatorId":2,"metricType":"sum"},{"name":"number of dynamic part","accumulatorId":3,"metricType":"sum"}]},"time":1622043425909} -{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1622043426367,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"json at QualificationInfoUtils.scala:76","Number of Tasks":12,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"Scan\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at QualificationInfoUtils.scala:64","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameWriter.json(DataFrameWriter.scala:853)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1(QualificationInfoUtils.scala:76)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1$adapted(QualificationInfoUtils.scala:73)\norg.apache.spark.rapids.TestUtils$.withTempPath(TestUtils.scala:31)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genDatasetEventLog(QualificationInfoUtils.scala:73)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:185)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[0],"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"1\",\"name\":\"Execute InsertIntoHadoopFsRelationCommand\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"0","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"json at QualificationInfoUtils.scala:76","Number of Tasks":12,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"Scan\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at QualificationInfoUtils.scala:64","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameWriter.json(DataFrameWriter.scala:853)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1(QualificationInfoUtils.scala:76)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1$adapted(QualificationInfoUtils.scala:73)\norg.apache.spark.rapids.TestUtils$.withTempPath(TestUtils.scala:31)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genDatasetEventLog(QualificationInfoUtils.scala:73)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:185)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043426387,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"1\",\"name\":\"Execute InsertIntoHadoopFsRelationCommand\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"0","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"}} +{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1622043426367,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"json at QualificationInfoUtils.scala:76","Number of Tasks":12,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"Scan\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at QualificationInfoUtils.scala:64","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameWriter.json(DataFrameWriter.scala:853)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1(QualificationInfoUtils.scala:76)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1$adapted(QualificationInfoUtils.scala:73)\norg.apache.spark.rapids.TestUtils$.withTempPath(TestUtils.scala:31)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genDatasetEventLog(QualificationInfoUtils.scala:73)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:185)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[0],"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"1\",\"name\":\"Execute InsertIntoHadoopFsRelationCommand\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"0","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423013"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"json at QualificationInfoUtils.scala:76","Number of Tasks":12,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"Scan\"}","Callsite":"json at QualificationInfoUtils.scala:76","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at QualificationInfoUtils.scala:64","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":12,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameWriter.json(DataFrameWriter.scala:853)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1(QualificationInfoUtils.scala:76)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.$anonfun$genDatasetEventLog$1$adapted(QualificationInfoUtils.scala:73)\norg.apache.spark.rapids.TestUtils$.withTempPath(TestUtils.scala:31)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genDatasetEventLog(QualificationInfoUtils.scala:73)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:185)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043426387,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"1\",\"name\":\"Execute InsertIntoHadoopFsRelationCommand\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"0","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423013"}} {"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1622043426546,"Executor ID":"driver","Host":"10.10.10.100","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} {"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1622043426567,"Executor ID":"driver","Host":"10.10.10.100","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} {"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1622043426568,"Executor ID":"driver","Host":"10.10.10.100","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} @@ -39,9 +39,9 @@ {"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":0,"accumUpdates":[[0,12],[1,16560],[2,720],[3,0]]} {"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":0,"time":1622043427215} {"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":1,"description":"count at QualificationInfoUtils.scala:94","details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\n* HashAggregate (15)\n+- Exchange (14)\n +- * HashAggregate (13)\n +- * Project (12)\n +- * SortMergeJoin Inner (11)\n :- * Sort (5)\n : +- Exchange (4)\n : +- * Project (3)\n : +- * SerializeFromObject (2)\n : +- Scan (1)\n +- * Sort (10)\n +- Exchange (9)\n +- * Project (8)\n +- * SerializeFromObject (7)\n +- Scan (6)\n\n\n(1) Scan\nOutput [1]: [obj#40]\nArguments: obj#40: int, ParallelCollectionRDD[5] at makeRDD at QualificationInfoUtils.scala:92\n\n(2) SerializeFromObject [codegen id : 1]\nInput [1]: [obj#40]\nArguments: [input[0, int, false] AS value#41]\n\n(3) Project [codegen id : 1]\nOutput [1]: [value#41 AS a#51]\nInput [1]: [value#41]\n\n(4) Exchange\nInput [1]: [a#51]\nArguments: hashpartitioning(a#51, 200), ENSURE_REQUIREMENTS, [id=#93]\n\n(5) Sort [codegen id : 2]\nInput [1]: [a#51]\nArguments: [a#51 ASC NULLS FIRST], false, 0\n\n(6) Scan\nOutput [1]: [obj#46]\nArguments: obj#46: int, ParallelCollectionRDD[6] at makeRDD at QualificationInfoUtils.scala:93\n\n(7) SerializeFromObject [codegen id : 3]\nInput [1]: [obj#46]\nArguments: [input[0, int, false] AS value#47]\n\n(8) Project [codegen id : 3]\nOutput [1]: [value#47 AS b#53]\nInput [1]: [value#47]\n\n(9) Exchange\nInput [1]: [b#53]\nArguments: hashpartitioning(b#53, 200), ENSURE_REQUIREMENTS, [id=#102]\n\n(10) Sort [codegen id : 4]\nInput [1]: [b#53]\nArguments: [b#53 ASC NULLS FIRST], false, 0\n\n(11) SortMergeJoin [codegen id : 5]\nLeft keys [1]: [a#51]\nRight keys [1]: [b#53]\nJoin condition: None\n\n(12) Project [codegen id : 5]\nOutput: []\nInput [2]: [a#51, b#53]\n\n(13) HashAggregate [codegen id : 5]\nInput: []\nKeys: []\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count#64L]\nResults [1]: [count#65L]\n\n(14) Exchange\nInput [1]: [count#65L]\nArguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#111]\n\n(15) HashAggregate [codegen id : 6]\nInput [1]: [count#65L]\nKeys: []\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#61L]\nResults [1]: [count(1)#61L AS count#62L]\n\n","sparkPlanInfo":{"nodeName":"WholeStageCodegen (6)","simpleString":"WholeStageCodegen (6)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Exchange","simpleString":"Exchange SinglePartition, ENSURE_REQUIREMENTS, [id=#111]","children":[{"nodeName":"WholeStageCodegen (5)","simpleString":"WholeStageCodegen (5)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[], functions=[partial_count(1)])","children":[{"nodeName":"Project","simpleString":"Project","children":[{"nodeName":"SortMergeJoin","simpleString":"SortMergeJoin [a#51], [b#53], Inner","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"Sort","simpleString":"Sort [a#51 ASC NULLS FIRST], false, 0","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(a#51, 200), ENSURE_REQUIREMENTS, [id=#93]","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [value#41 AS a#51]","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [input[0, int, false] AS value#41]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Scan","simpleString":"Scan[obj#40]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":116,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":115,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":74,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":75,"metricType":"nsTiming"},{"name":"records read","accumulatorId":72,"metricType":"sum"},{"name":"local bytes read","accumulatorId":70,"metricType":"size"},{"name":"fetch wait time","accumulatorId":71,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":68,"metricType":"size"},{"name":"local blocks read","accumulatorId":67,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":66,"metricType":"sum"},{"name":"data size","accumulatorId":65,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":69,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":73,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"sort time","accumulatorId":112,"metricType":"timing"},{"name":"peak memory","accumulatorId":113,"metricType":"size"},{"name":"spill size","accumulatorId":114,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":111,"metricType":"timing"}]}],"metadata":{},"metrics":[]},{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"Sort","simpleString":"Sort [b#53 ASC NULLS FIRST], false, 0","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(b#53, 200), ENSURE_REQUIREMENTS, [id=#102]","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"Project","simpleString":"Project [value#47 AS b#53]","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [input[0, int, false] AS value#47]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Scan","simpleString":"Scan[obj#46]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":122,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":121,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":85,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":86,"metricType":"nsTiming"},{"name":"records read","accumulatorId":83,"metricType":"sum"},{"name":"local bytes read","accumulatorId":81,"metricType":"size"},{"name":"fetch wait time","accumulatorId":82,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":79,"metricType":"size"},{"name":"local blocks read","accumulatorId":78,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":77,"metricType":"sum"},{"name":"data size","accumulatorId":76,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":80,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":84,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"sort time","accumulatorId":118,"metricType":"timing"},{"name":"peak memory","accumulatorId":119,"metricType":"size"},{"name":"spill size","accumulatorId":120,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":117,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":110,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":107,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":108,"metricType":"timing"},{"name":"peak memory","accumulatorId":106,"metricType":"size"},{"name":"number of output rows","accumulatorId":105,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":109,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":104,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":96,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":97,"metricType":"nsTiming"},{"name":"records read","accumulatorId":94,"metricType":"sum"},{"name":"local bytes read","accumulatorId":92,"metricType":"size"},{"name":"fetch wait time","accumulatorId":93,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":90,"metricType":"size"},{"name":"local blocks read","accumulatorId":89,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":88,"metricType":"sum"},{"name":"data size","accumulatorId":87,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":91,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":95,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":101,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":102,"metricType":"timing"},{"name":"peak memory","accumulatorId":100,"metricType":"size"},{"name":"number of output rows","accumulatorId":99,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":103,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":98,"metricType":"timing"}]},"time":1622043427414} -{"Event":"SparkListenerJobStart","Job ID":1,"Submission Time":1622043427637,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"43\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"15\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:93","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"39\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"14\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:92","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"29\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":200,"RDD Info":[{"RDD ID":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[18],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[14],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"20\",\"name\":\"WholeStageCodegen (5)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"35\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"25\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"20\",\"name\":\"WholeStageCodegen (5)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[11,16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[1,2],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":1,"RDD Info":[{"RDD ID":22,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"44\",\"name\":\"mapPartitionsInternal\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":21,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"16\",\"name\":\"WholeStageCodegen (6)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"19\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[3],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[1,2,3,4],"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"43\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"15\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:93","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"39\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043427644,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"14\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:92","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"29\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043427656,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423018"}} +{"Event":"SparkListenerJobStart","Job ID":1,"Submission Time":1622043427637,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"43\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"15\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:93","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"39\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"14\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:92","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"29\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":200,"RDD Info":[{"RDD ID":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[18],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[14],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"20\",\"name\":\"WholeStageCodegen (5)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"35\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"25\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"20\",\"name\":\"WholeStageCodegen (5)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[11,16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[1,2],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":1,"RDD Info":[{"RDD ID":22,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"44\",\"name\":\"mapPartitionsInternal\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":21,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"16\",\"name\":\"WholeStageCodegen (6)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"19\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[3],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[1,2,3,4],"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423013"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"38\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"43\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"15\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:93","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"39\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043427644,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423013"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at QualificationInfoUtils.scala:94","Number of Tasks":6,"RDD Info":[{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"28\",\"name\":\"Exchange\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"14\",\"name\":\"makeRDD\"}","Callsite":"makeRDD at QualificationInfoUtils.scala:92","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"29\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"Scan\"}","Callsite":"count at QualificationInfoUtils.scala:94","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.count(Dataset.scala:3005)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.genjoinDataFrameOpEventLog(QualificationInfoUtils.scala:94)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils$.main(QualificationInfoUtils.scala:186)\ncom.nvidia.spark.rapids.tool.profiling.QualificationInfoUtils.main(QualificationInfoUtils.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1622043427656,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.sql.warehouse.dir":"file:/home/user1/workspace/spark-rapids-another/spark-warehouse","spark.driver.host":"10.10.10.100","spark.eventLog.enabled":"true","spark.driver.port":"43493","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar,file:/home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar","spark.app.name":"Rapids Spark Profiling Tool Unit Tests","spark.rdd.scope":"{\"id\":\"45\",\"name\":\"collect\"}","spark.driver.memory":"30g","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.app.startTime":"1622043422343","spark.executor.id":"driver","spark.app.initial.jar.urls":"spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,spark://10.10.10.100:43493/jars/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/home/user1/testeventlogDir","spark.sql.execution.id":"1","spark.repl.local.jars":"file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT-tests.jar,file:///home/user1/workspace/spark-rapids-another/rapids-4-spark-tools/target/rapids-4-spark-tools-21.06.0-SNAPSHOT.jar","spark.app.id":"local-1622043423013"}} {"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":0,"Attempt":0,"Launch Time":1622043427655,"Executor ID":"driver","Host":"10.10.10.100","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} {"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":1,"Attempt":0,"Launch Time":1622043427661,"Executor ID":"driver","Host":"10.10.10.100","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} {"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":2,"Attempt":0,"Launch Time":1622043427661,"Executor ID":"driver","Host":"10.10.10.100","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} diff --git a/core/src/test/resources/spark-events-qualification/eventlog_same_app_id_1.zstd b/core/src/test/resources/spark-events-qualification/eventlog_same_app_id_1.zstd new file mode 100644 index 000000000..fc1dd90f7 Binary files /dev/null and b/core/src/test/resources/spark-events-qualification/eventlog_same_app_id_1.zstd differ diff --git a/core/src/test/resources/spark-events-qualification/eventlog_same_app_id_2.zstd b/core/src/test/resources/spark-events-qualification/eventlog_same_app_id_2.zstd new file mode 100644 index 000000000..fc1dd90f7 Binary files /dev/null and b/core/src/test/resources/spark-events-qualification/eventlog_same_app_id_2.zstd differ diff --git a/core/src/test/resources/spark-events-qualification/multiple_attempts/attempt_1_eventlog.zstd b/core/src/test/resources/spark-events-qualification/multiple_attempts/attempt_1_eventlog.zstd new file mode 100644 index 000000000..bb14d449c Binary files /dev/null and b/core/src/test/resources/spark-events-qualification/multiple_attempts/attempt_1_eventlog.zstd differ diff --git a/core/src/test/resources/spark-events-qualification/multiple_attempts/attempt_2_eventlog.zstd b/core/src/test/resources/spark-events-qualification/multiple_attempts/attempt_2_eventlog.zstd new file mode 100644 index 000000000..d6bd79947 Binary files /dev/null and b/core/src/test/resources/spark-events-qualification/multiple_attempts/attempt_2_eventlog.zstd differ diff --git a/core/src/test/resources/spark-events-qualification/multiple_attempts/attempt_3_eventlog.zstd b/core/src/test/resources/spark-events-qualification/multiple_attempts/attempt_3_eventlog.zstd new file mode 100644 index 000000000..cd7b640dc Binary files /dev/null and b/core/src/test/resources/spark-events-qualification/multiple_attempts/attempt_3_eventlog.zstd differ diff --git a/core/src/test/resources/spark-events-qualification/multiple_attempts/attempt_4_eventlog.zstd b/core/src/test/resources/spark-events-qualification/multiple_attempts/attempt_4_eventlog.zstd new file mode 100644 index 000000000..8cfba577a Binary files /dev/null and b/core/src/test/resources/spark-events-qualification/multiple_attempts/attempt_4_eventlog.zstd differ diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/AppFilterSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/AppFilterSuite.scala index 869e56acf..63e3265f3 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/AppFilterSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/AppFilterSuite.scala @@ -151,13 +151,13 @@ class AppFilterSuite extends BaseTestSuite { } val appsToTest = Array(TestEventLogInfo("ndshours18", msHoursAgo(18), 1), - TestEventLogInfo("ndsweeks2", msWeeksAgo(2), 1), - TestEventLogInfo("ndsmonths4", msMonthsAgo(5), 1), - TestEventLogInfo("ndsdays3", msDaysAgo(3), 1), - TestEventLogInfo("ndsmins34", msMinAgo(34), 1), - TestEventLogInfo("nds86", msDaysAgo(4), 1), - TestEventLogInfo("nds86", msWeeksAgo(2), 2), - TestEventLogInfo("otherapp", msWeeksAgo(2), 1)) + TestEventLogInfo("ndsweeks2", msWeeksAgo(2), 2), + TestEventLogInfo("ndsmonths4", msMonthsAgo(5), 3), + TestEventLogInfo("ndsdays3", msDaysAgo(3), 4), + TestEventLogInfo("ndsmins34", msMinAgo(34), 5), + TestEventLogInfo("nds86", msDaysAgo(4), 6), + TestEventLogInfo("nds86", msWeeksAgo(2), 7), + TestEventLogInfo("otherapp", msWeeksAgo(2), 8)) test("app name and start time 20m") { testTimePeriodAndStart(appsToTest, "20m", "nds", appsToTest.size - 1) @@ -225,9 +225,9 @@ class AppFilterSuite extends BaseTestSuite { private val appsWithFsToTest = Array( TestEventLogFSAndAppNameInfo("ndshours18", msHoursAgo(18), 1), - TestEventLogFSAndAppNameInfo("ndsweeks2", msWeeksAgo(2), 1), - TestEventLogFSAndAppNameInfo("nds86", msDaysAgo(4), 1), - TestEventLogFSAndAppNameInfo("nds86", msWeeksAgo(2), 2)) + TestEventLogFSAndAppNameInfo("ndsweeks2", msWeeksAgo(2), 2), + TestEventLogFSAndAppNameInfo("nds86", msDaysAgo(4), 3), + TestEventLogFSAndAppNameInfo("nds86", msWeeksAgo(2), 4)) test("app name exact and fs 10-newest-filesystem") { testFileSystemTimeAndStart(appsWithFsToTest, "10-newest-filesystem", "nds86", 2) @@ -356,9 +356,9 @@ class AppFilterSuite extends BaseTestSuite { private val appsFullWithFsToTest = Array( TestEventLogFSAndAppInfo("app-ndshours18", msHoursAgo(16), "ndshours18", msHoursAgo(18), 1), - TestEventLogFSAndAppInfo("app-ndsweeks2", msWeeksAgo(2), "ndsweeks2", msWeeksAgo(2), 1), - TestEventLogFSAndAppInfo("app-nds86-1", msDaysAgo(3), "nds86", msDaysAgo(4), 1), - TestEventLogFSAndAppInfo("app-nds86-2", msDaysAgo(13), "nds86", msWeeksAgo(2), 2)) + TestEventLogFSAndAppInfo("app-ndsweeks2", msWeeksAgo(2), "ndsweeks2", msWeeksAgo(2), 2), + TestEventLogFSAndAppInfo("app-nds86-1", msDaysAgo(3), "nds86", msDaysAgo(4), 3), + TestEventLogFSAndAppInfo("app-nds86-2", msDaysAgo(13), "nds86", msWeeksAgo(2), 4)) test("full app name exact and fs 10-newest-filesystem 6 days") { testFileSystemTimeAndStartAndAppFull(appsFullWithFsToTest, "10-newest-filesystem", @@ -460,37 +460,48 @@ class AppFilterSuite extends BaseTestSuite { private val appsWithAppNameCriteriaToTest = Array( TestEventLogFSAndAppInfo("app-ndshours18", msHoursAgo(16), "ndshours18", msHoursAgo(18), 1), - TestEventLogFSAndAppInfo("app-ndsweeks-1", msWeeksAgo(1), "ndsweeks", msWeeksAgo(1), 1), - TestEventLogFSAndAppInfo("app-ndsweeks-2", msWeeksAgo(2), "ndsweeks", msWeeksAgo(2), 2), - TestEventLogFSAndAppInfo("app-nds86-1", msDaysAgo(3), "nds86", msDaysAgo(4), 1), - TestEventLogFSAndAppInfo("app-nds86-2", msDaysAgo(13), "nds86", msWeeksAgo(2), 2), - TestEventLogFSAndAppInfo("app-nds86-3", msDaysAgo(18), "nds86", msWeeksAgo(3), 3)) + TestEventLogFSAndAppInfo("app-ndsweeks-1", msWeeksAgo(1), "ndsweeks", msWeeksAgo(1), 2), + TestEventLogFSAndAppInfo("app-ndsweeks-2", msWeeksAgo(2), "ndsweeks", msWeeksAgo(2), 3), + TestEventLogFSAndAppInfo("app-nds86-1", msDaysAgo(3), "nds86", msDaysAgo(4), 4), + TestEventLogFSAndAppInfo("app-nds86-2", msDaysAgo(13), "nds86", msWeeksAgo(2), 5), + TestEventLogFSAndAppInfo("app-nds86-3", msDaysAgo(18), "nds86", msWeeksAgo(3), 6)) test("standalone 1-oldest-per-app-name") { - val expected = Array(("ndshours18", "local-162610430031"), ("ndsweeks", "local-162610430032"), - ("nds86", "local-162610430033")) + val expected = Array( + ("ndshours18", "local-162610430031"), + ("ndsweeks", "local-162610430033"), + ("nds86", "local-162610430036")) testAppFilterCriteriaAndPerAppName(appsWithAppNameCriteriaToTest, "1-oldest-per-app-name", 3, expected) } test("standalone 2-newest-per-app-name") { - val expected = Array(("ndshours18", "local-162610430031"), ("ndsweeks", "local-162610430031"), - ("ndsweeks", "local-162610430032"), ("nds86", "local-162610430031"), - ("nds86", "local-162610430032")) + val expected = Array( + ("ndshours18", "local-162610430031"), + ("ndsweeks", "local-162610430032"), + ("ndsweeks", "local-162610430033"), + ("nds86", "local-162610430034"), + ("nds86", "local-162610430035")) testAppFilterCriteriaAndPerAppName(appsWithAppNameCriteriaToTest, "2-newest-per-app-name", 5, expected) } test("standalone 2-newest based on app time") { - val expected = Array(("ndshours18", "local-162610430031"), ("nds86", "local-162610430031")) + val expected = Array( + ("ndshours18", "local-162610430031"), + ("nds86", "local-162610430034")) testAppFilterCriteriaAndPerAppName(appsWithAppNameCriteriaToTest, "2-newest", 2, expected) } test("standalone 10-oldest based on app time") { - val expected = Array(("nds86", "local-162610430031"), ("nds86", "local-162610430032"), - ("nds86", "local-162610430033"), ("ndsweeks", "local-162610430031"), - ("ndsweeks", "local-162610430032"), ("ndshours18", "local-162610430031")) + val expected = Array( + ("ndshours18", "local-162610430031"), + ("ndsweeks", "local-162610430032"), + ("ndsweeks", "local-162610430033"), + ("nds86", "local-162610430034"), + ("nds86", "local-162610430035"), + ("nds86", "local-162610430036")) testAppFilterCriteriaAndPerAppName(appsWithAppNameCriteriaToTest, "10-oldest", 6, expected) } @@ -537,14 +548,14 @@ class AppFilterSuite extends BaseTestSuite { TestRegexAppNameAndUserName("app-ndshours18", msHoursAgo(16), "ndshours18", msHoursAgo(18), 1, "user1"), TestRegexAppNameAndUserName("app-ndsweeks-1", msWeeksAgo(1), "ndsweeks", - msWeeksAgo(1), 1, "user1"), + msWeeksAgo(1), 2, "user1"), TestRegexAppNameAndUserName("app-ndsweeks-2", msWeeksAgo(2), "ndsweeks", - msWeeksAgo(2), 2, "user2"), + msWeeksAgo(2), 3, "user2"), TestRegexAppNameAndUserName("app-ndsweeks-3", msWeeksAgo(3), "Ndsweeks", - msWeeksAgo(3), 3, "user3"), - TestRegexAppNameAndUserName("app-nds86-1", msDaysAgo(3), "nds86", msDaysAgo(4), 1, "user1"), - TestRegexAppNameAndUserName("app-nds86-2", msDaysAgo(13), "Nds86", msWeeksAgo(2), 2, "user2"), - TestRegexAppNameAndUserName("app-nds86-3", msDaysAgo(18), "nds86", msWeeksAgo(3), 3, "user3")) + msWeeksAgo(3), 4, "user3"), + TestRegexAppNameAndUserName("app-nds86-1", msDaysAgo(3), "nds86", msDaysAgo(4), 5, "user1"), + TestRegexAppNameAndUserName("app-nds86-2", msDaysAgo(13), "Nds86", msWeeksAgo(2), 6, "user2"), + TestRegexAppNameAndUserName("app-nds86-3", msDaysAgo(18), "nds86", msWeeksAgo(3), 7, "user3")) test("App Name Regex match with all user name") { testAppNameRegexAndUserName(appsWithAppNameRegexAndUserNameToTest, @@ -625,17 +636,17 @@ class AppFilterSuite extends BaseTestSuite { TestConjunctionAndDisjunction("app-ndshours18", msHoursAgo(16), "Ndshours18", msHoursAgo(18), 1, "user1"), TestConjunctionAndDisjunction("app-Ndsweeks-1", msWeeksAgo(1), "ndsweeks", - msWeeksAgo(1), 1, "user1"), + msWeeksAgo(1), 2, "user1"), TestConjunctionAndDisjunction("app-ndsweeks-2", msWeeksAgo(2), "Ndsweeks", - msWeeksAgo(2), 2, "user2"), + msWeeksAgo(2), 3, "user2"), TestConjunctionAndDisjunction("app-ndsweeks-3", msWeeksAgo(3), "ndsweeks", - msWeeksAgo(3), 3, "user3"), + msWeeksAgo(3), 4, "user3"), TestConjunctionAndDisjunction("app-Nds86-1", msDaysAgo(3), "nds86", - msDaysAgo(4), 1, "user1"), + msDaysAgo(4), 5, "user1"), TestConjunctionAndDisjunction("app-nds86-2", msDaysAgo(6), "nds86", - msWeeksAgo(1), 2, "user2"), + msWeeksAgo(1), 6, "user2"), TestConjunctionAndDisjunction("app-nds86-3", msDaysAgo(18), "nds86", - msWeeksAgo(3), 3, "user3")) + msWeeksAgo(3), 7, "user3")) test("Test disjunction all filters") { testConjunctionAndDisjunction(appsNameConjunctionAndDisjunctionToTest, @@ -846,12 +857,14 @@ class AppFilterSuite extends BaseTestSuite { TrampolineUtil.withTempDir { tmpEventLogDir => val fileNames = apps.map { app => + val userPattern = "user(\\d+)".r + val userId = userPattern.findFirstMatchIn(app.userName).get.group(1).toInt val elogFile = Paths.get(tmpEventLogDir.getAbsolutePath, app.fileName) // scalastyle:off line.size.limit val supText = s"""{"Event":"SparkListenerLogStart","Spark Version":"3.1.1"} |{"Event":"SparkListenerApplicationStart","App Name":"${app.appName}", "App ID":"local-16261043003${app.uniqueId}","Timestamp":${app.appTime}, "User":"${app.userName}"} - |{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/lib/jvm/java-8-openjdk-amd64/jre"},"Spark Properties":{"spark.driver.host":"10.10.19.1${app.uniqueId}","spark.app.name":"${app.appName}","spark.driver.port":"4349${app.uniqueId}","spark.eventLog.enabled":"true","spark.master":"spark://5.6.7.8:707${app.uniqueId + 4}","spark.redaction.regex":"*********(redacted)","spark.eventLog.dir":"file:///tmp/spark-events-${app.uniqueId}","spark.sql.maven.additionalRemoteRepositories":"https://maven-central.storage-download.googleapis.com/maven2/","spark.sql.hive.metastore.sharedPrefixes":"com.mysql.jdbc,org.postgresql,com.microsoft.sqlserver","spark.shuffle.io.maxRetries":"${app.uniqueId}","spark.shuffle.registration.maxAttempts":"${app.uniqueId + 1}"},"Hadoop Properties":{"hadoop.service.shutdown.timeout":"30s"},"System Properties":{"java.io.tmpdir":"/tmp"},"Classpath Entries":{"/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-exec-2.3.7-core.jar":"System Classpath"}}""".stripMargin + |{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/lib/jvm/java-8-openjdk-amd64/jre"},"Spark Properties":{"spark.driver.host":"10.10.19.1$userId","spark.app.name":"${app.appName}","spark.driver.port":"4349$userId","spark.eventLog.enabled":"true","spark.master":"spark://5.6.7.8:707${userId + 4}","spark.redaction.regex":"*********(redacted)","spark.eventLog.dir":"file:///tmp/spark-events-$userId","spark.sql.maven.additionalRemoteRepositories":"https://maven-central.storage-download.googleapis.com/maven2/","spark.sql.hive.metastore.sharedPrefixes":"com.mysql.jdbc,org.postgresql,com.microsoft.sqlserver","spark.shuffle.io.maxRetries":"$userId","spark.shuffle.registration.maxAttempts":"${userId + 1}"},"Hadoop Properties":{"hadoop.service.shutdown.timeout":"30s"},"System Properties":{"java.io.tmpdir":"/tmp"},"Classpath Entries":{"/home/user1/runspace/spark311/spark-3.1.1-bin-hadoop3.2/jars/hive-exec-2.3.7-core.jar":"System Classpath"}}""".stripMargin // scalastyle:on line.size.limit Files.write(elogFile, supText.getBytes(StandardCharsets.UTF_8)) new File(elogFile.toString).setLastModified(app.fsTime) @@ -871,7 +884,7 @@ class AppFilterSuite extends BaseTestSuite { } } } - + test("Test filtering eventlog with missing start event") { TrampolineUtil.withTempDir { outpath => TrampolineUtil.withTempDir { tmpEventLogDir => diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/QualificationSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/QualificationSuite.scala index 547f0f310..d1d4eecad 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/QualificationSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/QualificationSuite.scala @@ -533,11 +533,13 @@ class QualificationSuite extends BaseTestSuite { val allArgs = Array( "--output-directory", outpath.getAbsolutePath()) - - val appArgs = new QualificationArgs(allArgs ++ logFiles) - val (exit, appSum) = QualificationMain.mainInternal(appArgs) - assert(exit == 0) - assert(appSum.size == pwList.length + 1) + // test qualification one file at a time to avoid merging results as a single app + for (logFile <- logFiles) { + val appArgs = new QualificationArgs(allArgs ++ Array(logFile)) + val (exit, appSum) = QualificationMain.mainInternal(appArgs) + assert(exit == 0) + assert(appSum.size == 1) + } // test Profiler val apps = ToolTestUtils.processProfileApps(logFiles, sparkSession) assert(apps.size == pwList.length + 1) @@ -1750,6 +1752,57 @@ class QualificationSuite extends BaseTestSuite { val expectedStatus = Some(StatusReportCounts(2, 1, 0, 0)) runQualificationTest(logFiles, expectedStatus = expectedStatus) } + + test("process multiple attempts of the same app ID and skip lower attempts") { + TrampolineUtil.withTempDir { outPath => + val baseArgs = Array("--output-directory", + outPath.getAbsolutePath, + "-n", "12", + s"$logDir/multiple_attempts/*") + val appArgs = new QualificationArgs(baseArgs) + val (exitCode, result) = QualificationMain.mainInternal(appArgs) + assert(exitCode == 0 && result.size == 1, + "Qualification tool returned unexpected results.") + + val statusResultFile = s"$outPath/${QualOutputWriter.LOGFILE_NAME}/" + + s"${QualOutputWriter.LOGFILE_NAME}_status.csv" + + // Verify that the status file contains the expected messages for skipped + // attempts (1, 2, 3) and thus only the latest attempt (4) is processed. + val statusFileContents = UTF8Source.fromFile(statusResultFile).mkString + Seq(1, 2, 3).foreach { attemptId => + val expectedMessage = s"skipping this attempt $attemptId as a newer " + + "attemptId is being processed" + assert(statusFileContents.contains(expectedMessage), + s"Expected message not found in status file: $expectedMessage") + } + + // Status counts: 1 SUCCESS, 0 FAILURE, 3 SKIPPED, 0 UNKNOWN + val expectedStatusCount = StatusReportCounts(1, 0, 3, 0) + ToolTestUtils.compareStatusReport(sparkSession, expectedStatusCount, statusResultFile) + } + } + + ignore("process multiple event logs with same app ID and attempt ID: Not supported") { + TrampolineUtil.withTempDir { outPath => + val baseArgs = Array("--output-directory", + outPath.getAbsolutePath, + s"$logDir/eventlog_same_app_id_1.zstd", + s"$logDir/eventlog_same_app_id_2.zstd") + val appArgs = new QualificationArgs(baseArgs) + val (exitCode, result) = QualificationMain.mainInternal(appArgs) + assert(exitCode == 0 && result.size == 1, + "Qualification tool returned unexpected results.") + + val statusResultFile = s"$outPath/${QualOutputWriter.LOGFILE_NAME}/" + + s"${QualOutputWriter.LOGFILE_NAME}_status.csv" + + // Only one of the event logs should be processed and the other should be skipped. + // Status counts: 1 SUCCESS, 0 FAILURE, 1 SKIPPED, 0 UNKNOWN + val expectedStatusCount = StatusReportCounts(1, 0, 1, 0) + ToolTestUtils.compareStatusReport(sparkSession, expectedStatusCount, statusResultFile) + } + } } class ToolTestListener extends SparkListener {