diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 9ecf316beeaa1..78f509c670839 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -83,7 +83,7 @@ class SparkContext(config: SparkConf) extends Logging { // The call site where this SparkContext was constructed. private val creationSite: CallSite = Utils.getCallSite() - if (!config.get(ALLOW_SPARK_CONTEXT_IN_EXECUTORS)) { + if (!config.get(EXECUTOR_ALLOW_SPARK_CONTEXT)) { // In order to prevent SparkContext from being created in executors. SparkContext.assertOnDriver() } diff --git a/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala b/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala index b98c7436f9906..a295ef06a6376 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala @@ -275,7 +275,8 @@ private[spark] object ConfigEntry { val UNDEFINED = "" - private val knownConfigs = new java.util.concurrent.ConcurrentHashMap[String, ConfigEntry[_]]() + private[spark] val knownConfigs = + new java.util.concurrent.ConcurrentHashMap[String, ConfigEntry[_]]() def registerEntry(entry: ConfigEntry[_]): Unit = { val existing = knownConfigs.putIfAbsent(entry.key, entry) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index fdc9253ce9b02..200cde0a2d3ed 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -1909,8 +1909,8 @@ package object config { .booleanConf .createWithDefault(false) - private[spark] val ALLOW_SPARK_CONTEXT_IN_EXECUTORS = - ConfigBuilder("spark.driver.allowSparkContextInExecutors") + private[spark] val EXECUTOR_ALLOW_SPARK_CONTEXT = + ConfigBuilder("spark.executor.allowSparkContext") .doc("If set to true, SparkContext can be created in executors.") .version("3.0.1") .booleanConf diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala index f7b0e9b62fc29..7ae9117137caa 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -24,7 +24,7 @@ import scala.collection.JavaConverters._ import scala.collection.mutable.HashMap import org.apache.spark._ -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.Logging import org.apache.spark.internal.config.CPUS_PER_TASK import org.apache.spark.internal.config.Status._ @@ -868,13 +868,17 @@ private[spark] class AppStatusListener( // check if there is a new peak value for any of the executor level memory metrics // for the live UI. SparkListenerExecutorMetricsUpdate events are only processed // for the live UI. - event.executorUpdates.foreach { case (_, peakUpdates) => + event.executorUpdates.foreach { case (key, peakUpdates) => liveExecutors.get(event.execId).foreach { exec => if (exec.peakExecutorMetrics.compareAndUpdatePeakValues(peakUpdates)) { - maybeUpdate(exec, now) + update(exec, now) } } + + // Update stage level peak executor metrics. + updateStageLevelPeakExecutorMetrics(key._1, key._2, event.execId, peakUpdates, now) } + // Flush updates if necessary. Executor heartbeat is an event that happens periodically. Flush // here to ensure the staleness of Spark UI doesn't last more than // `max(heartbeat interval, liveUpdateMinFlushPeriod)`. @@ -885,17 +889,38 @@ private[spark] class AppStatusListener( } } - override def onStageExecutorMetrics(executorMetrics: SparkListenerStageExecutorMetrics): Unit = { + override def onStageExecutorMetrics(event: SparkListenerStageExecutorMetrics): Unit = { val now = System.nanoTime() // check if there is a new peak value for any of the executor level memory metrics, // while reading from the log. SparkListenerStageExecutorMetrics are only processed // when reading logs. - liveExecutors.get(executorMetrics.execId).orElse( - deadExecutors.get(executorMetrics.execId)).foreach { exec => - if (exec.peakExecutorMetrics.compareAndUpdatePeakValues(executorMetrics.executorMetrics)) { - update(exec, now) - } + liveExecutors.get(event.execId).orElse( + deadExecutors.get(event.execId)).foreach { exec => + if (exec.peakExecutorMetrics.compareAndUpdatePeakValues(event.executorMetrics)) { + update(exec, now) + } + } + + // Update stage level peak executor metrics. + updateStageLevelPeakExecutorMetrics( + event.stageId, event.stageAttemptId, event.execId, event.executorMetrics, now) + } + + private def updateStageLevelPeakExecutorMetrics( + stageId: Int, + stageAttemptId: Int, + executorId: String, + executorMetrics: ExecutorMetrics, + now: Long): Unit = { + Option(liveStages.get((stageId, stageAttemptId))).foreach { stage => + if (stage.peakExecutorMetrics.compareAndUpdatePeakValues(executorMetrics)) { + update(stage, now) + } + val esummary = stage.executorSummary(executorId) + if (esummary.peakExecutorMetrics.compareAndUpdatePeakValues(executorMetrics)) { + update(esummary, now) + } } } diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala index 0a8d188dc1553..5c6543fe28a18 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala @@ -506,7 +506,8 @@ private[spark] class AppStatusStore( tasks = Some(tasks), executorSummary = Some(executorSummary(stage.stageId, stage.attemptId)), killedTasksSummary = stage.killedTasksSummary, - resourceProfileId = stage.resourceProfileId) + resourceProfileId = stage.resourceProfileId, + peakExecutorMetrics = stage.peakExecutorMetrics) } def rdd(rddId: Int): v1.RDDStorageInfo = { diff --git a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala index 81478214994b0..0fadd330a01ad 100644 --- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala +++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala @@ -365,6 +365,8 @@ private class LiveExecutorStageSummary( var metrics = createMetrics(default = 0L) + val peakExecutorMetrics = new ExecutorMetrics() + override protected def doUpdate(): Any = { val info = new v1.ExecutorStageSummary( taskTime, @@ -381,7 +383,8 @@ private class LiveExecutorStageSummary( metrics.shuffleWriteMetrics.recordsWritten, metrics.memoryBytesSpilled, metrics.diskBytesSpilled, - isBlacklisted) + isBlacklisted, + Some(peakExecutorMetrics).filter(_.isSet)) new ExecutorStageSummaryWrapper(stageId, attemptId, executorId, info) } @@ -420,6 +423,8 @@ private class LiveStage extends LiveEntity { var blackListedExecutors = new HashSet[String]() + val peakExecutorMetrics = new ExecutorMetrics() + // Used for cleanup of tasks after they reach the configured limit. Not written to the store. @volatile var cleaning = false var savedTasks = new AtomicInteger(0) @@ -484,7 +489,8 @@ private class LiveStage extends LiveEntity { tasks = None, executorSummary = None, killedTasksSummary = killedSummary, - resourceProfileId = info.resourceProfileId) + resourceProfileId = info.resourceProfileId, + Some(peakExecutorMetrics).filter(_.isSet)) } override protected def doUpdate(): Any = { diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala index e89e29101a126..d207a6023f7f9 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala @@ -82,7 +82,10 @@ class ExecutorStageSummary private[spark]( val shuffleWriteRecords : Long, val memoryBytesSpilled : Long, val diskBytesSpilled : Long, - val isBlacklistedForStage: Boolean) + val isBlacklistedForStage: Boolean, + @JsonSerialize(using = classOf[ExecutorMetricsJsonSerializer]) + @JsonDeserialize(using = classOf[ExecutorMetricsJsonDeserializer]) + val peakMemoryMetrics: Option[ExecutorMetrics]) class ExecutorSummary private[spark]( val id: String, @@ -259,7 +262,10 @@ class StageData private[spark]( val tasks: Option[Map[Long, TaskData]], val executorSummary: Option[Map[String, ExecutorStageSummary]], val killedTasksSummary: Map[String, Int], - val resourceProfileId: Int) + val resourceProfileId: Int, + @JsonSerialize(using = classOf[ExecutorMetricsJsonSerializer]) + @JsonDeserialize(using = classOf[ExecutorMetricsJsonDeserializer]) + val peakExecutorMetrics: Option[ExecutorMetrics]) class TaskData private[spark]( val taskId: Long, diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index f1962ef39fc06..2a3597e323543 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -17,7 +17,7 @@ package org.apache.spark.ui -import java.net.{URI, URL} +import java.net.{URI, URL, URLDecoder} import java.util.EnumSet import javax.servlet.DispatcherType import javax.servlet.http._ @@ -377,8 +377,7 @@ private[spark] object JettyUtils extends Logging { if (baseRequest.isSecure) { return } - val httpsURI = createRedirectURI(scheme, baseRequest.getServerName, securePort, - baseRequest.getRequestURI, baseRequest.getQueryString) + val httpsURI = createRedirectURI(scheme, securePort, baseRequest) response.setContentLength(0) response.sendRedirect(response.encodeRedirectURL(httpsURI)) baseRequest.setHandled(true) @@ -440,16 +439,34 @@ private[spark] object JettyUtils extends Logging { handler.addFilter(holder, "/*", EnumSet.allOf(classOf[DispatcherType])) } + private def decodeURL(url: String, encoding: String): String = { + if (url == null) { + null + } else { + URLDecoder.decode(url, encoding) + } + } + // Create a new URI from the arguments, handling IPv6 host encoding and default ports. - private def createRedirectURI( - scheme: String, server: String, port: Int, path: String, query: String) = { + private def createRedirectURI(scheme: String, port: Int, request: Request): String = { + val server = request.getServerName val redirectServer = if (server.contains(":") && !server.startsWith("[")) { s"[${server}]" } else { server } val authority = s"$redirectServer:$port" - new URI(scheme, authority, path, query, null).toString + val queryEncoding = if (request.getQueryEncoding != null) { + request.getQueryEncoding + } else { + // By default decoding the URI as "UTF-8" should be enough for SparkUI + "UTF-8" + } + // The request URL can be raw or encoded here. To avoid the request URL being + // encoded twice, let's decode it here. + val requestURI = decodeURL(request.getRequestURI, queryEncoding) + val queryString = decodeURL(request.getQueryString, queryEncoding) + new URI(scheme, authority, requestURI, queryString, null).toString } def toVirtualHosts(connectors: String*): Array[String] = connectors.map("@" + _).toArray diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala index bba5e3dda6c47..df239d6d0e187 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala @@ -255,7 +255,8 @@ private[ui] class JobPage(parent: JobsTab, store: AppStatusStore) extends WebUIP tasks = None, executorSummary = None, killedTasksSummary = Map(), - ResourceProfile.UNKNOWN_RESOURCE_PROFILE_ID) + ResourceProfile.UNKNOWN_RESOURCE_PROFILE_ID, + peakExecutorMetrics = None) } } diff --git a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json index d2b3d1b069204..06015ec46e44d 100644 --- a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json @@ -1,4 +1,19 @@ [ { + "id" : "app-20200706201101-0003", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2020-07-07T03:11:00.235GMT", + "endTime" : "2020-07-07T03:17:04.231GMT", + "lastUpdated" : "", + "duration" : 363996, + "sparkUser" : "terryk", + "completed" : true, + "appSparkVersion" : "3.1.0-SNAPSHOT", + "endTimeEpoch" : 1594091824231, + "startTimeEpoch" : 1594091460235, + "lastUpdatedEpoch" : 0 + } ] +}, { "id" : "application_1578436911597_0052", "name" : "Spark shell", "attempts" : [ { diff --git a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json index d2b3d1b069204..06015ec46e44d 100644 --- a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json @@ -1,4 +1,19 @@ [ { + "id" : "app-20200706201101-0003", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2020-07-07T03:11:00.235GMT", + "endTime" : "2020-07-07T03:17:04.231GMT", + "lastUpdated" : "", + "duration" : 363996, + "sparkUser" : "terryk", + "completed" : true, + "appSparkVersion" : "3.1.0-SNAPSHOT", + "endTimeEpoch" : 1594091824231, + "startTimeEpoch" : 1594091460235, + "lastUpdatedEpoch" : 0 + } ] +}, { "id" : "application_1578436911597_0052", "name" : "Spark shell", "attempts" : [ { diff --git a/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json index 82489e94a84c8..8e6be68b4670c 100644 --- a/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json @@ -1,4 +1,19 @@ [ { + "id" : "app-20200706201101-0003", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2020-07-07T03:11:00.235GMT", + "endTime" : "2020-07-07T03:17:04.231GMT", + "lastUpdated" : "", + "duration" : 363996, + "sparkUser" : "terryk", + "completed" : true, + "appSparkVersion" : "3.1.0-SNAPSHOT", + "lastUpdatedEpoch" : 0, + "endTimeEpoch" : 1594091824231, + "startTimeEpoch" : 1594091460235 + } ] +}, { "id" : "application_1578436911597_0052", "name" : "Spark shell", "attempts" : [ { @@ -28,19 +43,4 @@ "lastUpdatedEpoch" : 0, "endTimeEpoch" : 1562101355974 } ] -}, { - "id" : "application_1553914137147_0018", - "name" : "LargeBlocks", - "attempts" : [ { - "startTime" : "2019-04-08T20:39:44.286GMT", - "endTime" : "2019-04-08T20:40:46.454GMT", - "lastUpdated" : "", - "duration" : 62168, - "sparkUser" : "systest", - "completed" : true, - "appSparkVersion" : "3.0.0-SNAPSHOT", - "startTimeEpoch" : 1554755984286, - "endTimeEpoch" : 1554756046454, - "lastUpdatedEpoch" : 0 - } ] } ] diff --git a/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json index ac2bb0e29b2fb..35d71f9d4409b 100644 --- a/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json @@ -1,4 +1,19 @@ [ { + "id" : "app-20200706201101-0003", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2020-07-07T03:11:00.235GMT", + "endTime" : "2020-07-07T03:17:04.231GMT", + "lastUpdated" : "", + "duration" : 363996, + "sparkUser" : "terryk", + "completed" : true, + "appSparkVersion" : "3.1.0-SNAPSHOT", + "endTimeEpoch" : 1594091824231, + "startTimeEpoch" : 1594091460235, + "lastUpdatedEpoch" : 0 + } ] +}, { "id" : "application_1578436911597_0052", "name" : "Spark shell", "attempts" : [ { @@ -14,22 +29,20 @@ "lastUpdatedEpoch" : 0 } ] }, { - "id": "application_1555004656427_0144", - "name": "Spark shell", - "attempts": [ - { - "startTime": "2019-07-02T21:02:17.180GMT", - "endTime": "2019-07-02T21:02:35.974GMT", - "lastUpdated": "", - "duration": 18794, - "sparkUser": "tgraves", - "completed": true, - "appSparkVersion": "3.0.0-SNAPSHOT", - "startTimeEpoch": 1562101337180, - "lastUpdatedEpoch": 0, - "endTimeEpoch": 1562101355974 - } - ] + "id" : "application_1555004656427_0144", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2019-07-02T21:02:17.180GMT", + "endTime" : "2019-07-02T21:02:35.974GMT", + "lastUpdated" : "", + "duration" : 18794, + "sparkUser" : "tgraves", + "completed" : true, + "appSparkVersion" : "3.0.0-SNAPSHOT", + "endTimeEpoch" : 1562101355974, + "startTimeEpoch" : 1562101337180, + "lastUpdatedEpoch" : 0 + } ] }, { "id" : "application_1553914137147_0018", "name" : "LargeBlocks", diff --git a/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json index 156167606ff20..c6530b14bf271 100644 --- a/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json @@ -1,4 +1,19 @@ [ { + "id" : "app-20200706201101-0003", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2020-07-07T03:11:00.235GMT", + "endTime" : "2020-07-07T03:17:04.231GMT", + "lastUpdated" : "", + "duration" : 363996, + "sparkUser" : "terryk", + "completed" : true, + "appSparkVersion" : "3.1.0-SNAPSHOT", + "endTimeEpoch" : 1594091824231, + "startTimeEpoch" : 1594091460235, + "lastUpdatedEpoch" : 0 + } ] +}, { "id" : "application_1578436911597_0052", "name" : "Spark shell", "attempts" : [ { diff --git a/core/src/test/resources/HistoryServerExpectations/stage_list_with_peak_metrics_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_list_with_peak_metrics_expectation.json new file mode 100644 index 0000000000000..d455b97fdafa4 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/stage_list_with_peak_metrics_expectation.json @@ -0,0 +1,204 @@ +[ { + "status" : "COMPLETE", + "stageId" : 2, + "attemptId" : 0, + "numTasks" : 16, + "numActiveTasks" : 0, + "numCompleteTasks" : 16, + "numFailedTasks" : 0, + "numKilledTasks" : 0, + "numCompletedIndices" : 16, + "submissionTime" : "2020-07-07T03:11:21.040GMT", + "firstTaskLaunchedTime" : "2020-07-07T03:11:21.077GMT", + "completionTime" : "2020-07-07T03:11:23.044GMT", + "executorDeserializeTime" : 3905, + "executorDeserializeCpuTime" : 979900000, + "executorRunTime" : 25579, + "executorCpuTime" : 8810338000, + "resultSize" : 33883, + "jvmGcTime" : 1010, + "resultSerializationTime" : 11, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 384640, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleRemoteBlocksFetched" : 0, + "shuffleLocalBlocksFetched" : 0, + "shuffleFetchWaitTime" : 0, + "shuffleRemoteBytesRead" : 0, + "shuffleRemoteBytesReadToDisk" : 0, + "shuffleLocalBytesRead" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 0, + "shuffleWriteTime" : 0, + "shuffleWriteRecords" : 0, + "name" : "foreach at :26", + "details" : "org.apache.spark.sql.Dataset.foreach(Dataset.scala:2862)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:26)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:30)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:32)\n$line19.$read$$iw$$iw$$iw$$iw$$iw.(:34)\n$line19.$read$$iw$$iw$$iw$$iw.(:36)\n$line19.$read$$iw$$iw$$iw.(:38)\n$line19.$read$$iw$$iw.(:40)\n$line19.$read$$iw.(:42)\n$line19.$read.(:44)\n$line19.$read$.(:48)\n$line19.$read$.()\n$line19.$eval$.$print$lzycompute(:7)\n$line19.$eval$.$print(:6)\n$line19.$eval.$print()\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)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:745)", + "schedulingPool" : "default", + "rddIds" : [ 10, 8, 6, 7, 9 ], + "accumulatorUpdates" : [ ], + "killedTasksSummary" : { }, + "resourceProfileId" : 0, + "peakExecutorMetrics" : { + "JVMHeapMemory" : 213367864, + "JVMOffHeapMemory" : 189011656, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 2133349, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 2133349, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 282024, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 0, + "ProcessTreeJVMRSSMemory" : 0, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 13, + "MinorGCTime" : 115, + "MajorGCCount" : 4, + "MajorGCTime" : 339 + } +}, { + "status" : "COMPLETE", + "stageId" : 1, + "attemptId" : 0, + "numTasks" : 16, + "numActiveTasks" : 0, + "numCompleteTasks" : 16, + "numFailedTasks" : 0, + "numKilledTasks" : 0, + "numCompletedIndices" : 16, + "submissionTime" : "2020-07-07T03:11:20.499GMT", + "firstTaskLaunchedTime" : "2020-07-07T03:11:20.502GMT", + "completionTime" : "2020-07-07T03:11:20.930GMT", + "executorDeserializeTime" : 424, + "executorDeserializeCpuTime" : 63666000, + "executorRunTime" : 6105, + "executorCpuTime" : 426449000, + "resultSize" : 31546, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 16256, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleRemoteBlocksFetched" : 0, + "shuffleLocalBlocksFetched" : 0, + "shuffleFetchWaitTime" : 0, + "shuffleRemoteBytesRead" : 0, + "shuffleRemoteBytesReadToDisk" : 0, + "shuffleLocalBytesRead" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 0, + "shuffleWriteTime" : 0, + "shuffleWriteRecords" : 0, + "name" : "$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266", + "description" : "broadcast exchange (runId bac7289a-c1d8-4966-a6a3-d9f347e13a5d)", + "details" : "org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:185)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)", + "schedulingPool" : "default", + "rddIds" : [ 5, 3, 4 ], + "accumulatorUpdates" : [ ], + "killedTasksSummary" : { }, + "resourceProfileId" : 0, + "peakExecutorMetrics" : { + "JVMHeapMemory" : 0, + "JVMOffHeapMemory" : 0, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 0, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 0, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 0, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 0, + "ProcessTreeJVMRSSMemory" : 0, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 0, + "MinorGCTime" : 0, + "MajorGCCount" : 0, + "MajorGCTime" : 0 + } +}, { + "status" : "COMPLETE", + "stageId" : 0, + "attemptId" : 0, + "numTasks" : 16, + "numActiveTasks" : 0, + "numCompleteTasks" : 16, + "numFailedTasks" : 0, + "numKilledTasks" : 0, + "numCompletedIndices" : 16, + "submissionTime" : "2020-07-07T03:11:18.860GMT", + "firstTaskLaunchedTime" : "2020-07-07T03:11:19.253GMT", + "completionTime" : "2020-07-07T03:11:20.381GMT", + "executorDeserializeTime" : 15399, + "executorDeserializeCpuTime" : 3668889000, + "executorRunTime" : 1292, + "executorCpuTime" : 172863000, + "resultSize" : 22375, + "jvmGcTime" : 1744, + "resultSerializationTime" : 3, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleRemoteBlocksFetched" : 0, + "shuffleLocalBlocksFetched" : 0, + "shuffleFetchWaitTime" : 0, + "shuffleRemoteBytesRead" : 0, + "shuffleRemoteBytesReadToDisk" : 0, + "shuffleLocalBytesRead" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 0, + "shuffleWriteTime" : 0, + "shuffleWriteRecords" : 0, + "name" : "$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266", + "description" : "broadcast exchange (runId bb0234e5-4157-49a4-b40c-6d538d9f2ec8)", + "details" : "org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:185)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)", + "schedulingPool" : "default", + "rddIds" : [ 2, 0, 1 ], + "accumulatorUpdates" : [ ], + "killedTasksSummary" : { }, + "resourceProfileId" : 0, + "peakExecutorMetrics" : { + "JVMHeapMemory" : 155100856, + "JVMOffHeapMemory" : 64239224, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 6964, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 6964, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 1852, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 0, + "ProcessTreeJVMRSSMemory" : 0, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 7, + "MinorGCTime" : 33, + "MajorGCCount" : 3, + "MajorGCTime" : 110 + } +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_with_peak_metrics_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_with_peak_metrics_expectation.json new file mode 100644 index 0000000000000..373510d23058e --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/stage_with_peak_metrics_expectation.json @@ -0,0 +1,998 @@ +{ + "status" : "COMPLETE", + "stageId" : 2, + "attemptId" : 0, + "numTasks" : 16, + "numActiveTasks" : 0, + "numCompleteTasks" : 16, + "numFailedTasks" : 0, + "numKilledTasks" : 0, + "numCompletedIndices" : 16, + "submissionTime" : "2020-07-07T03:11:21.040GMT", + "firstTaskLaunchedTime" : "2020-07-07T03:11:21.077GMT", + "completionTime" : "2020-07-07T03:11:23.044GMT", + "executorDeserializeTime" : 3905, + "executorDeserializeCpuTime" : 979900000, + "executorRunTime" : 25579, + "executorCpuTime" : 8810338000, + "resultSize" : 33883, + "jvmGcTime" : 1010, + "resultSerializationTime" : 11, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 384640, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleRemoteBlocksFetched" : 0, + "shuffleLocalBlocksFetched" : 0, + "shuffleFetchWaitTime" : 0, + "shuffleRemoteBytesRead" : 0, + "shuffleRemoteBytesReadToDisk" : 0, + "shuffleLocalBytesRead" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 0, + "shuffleWriteTime" : 0, + "shuffleWriteRecords" : 0, + "name" : "foreach at :26", + "details" : "org.apache.spark.sql.Dataset.foreach(Dataset.scala:2862)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:26)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:30)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:32)\n$line19.$read$$iw$$iw$$iw$$iw$$iw.(:34)\n$line19.$read$$iw$$iw$$iw$$iw.(:36)\n$line19.$read$$iw$$iw$$iw.(:38)\n$line19.$read$$iw$$iw.(:40)\n$line19.$read$$iw.(:42)\n$line19.$read.(:44)\n$line19.$read$.(:48)\n$line19.$read$.()\n$line19.$eval$.$print$lzycompute(:7)\n$line19.$eval$.$print(:6)\n$line19.$eval.$print()\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)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:745)", + "schedulingPool" : "default", + "rddIds" : [ 10, 8, 6, 7, 9 ], + "accumulatorUpdates" : [ ], + "tasks" : { + "42" : { + "taskId" : 42, + "index" : 10, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.120GMT", + "duration" : 1923, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 229, + "executorDeserializeCpuTime" : 58152000, + "executorRunTime" : 1624, + "executorCpuTime" : 508230000, + "resultSize" : 2115, + "jvmGcTime" : 66, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 70, + "gettingResultTime" : 0 + }, + "37" : { + "taskId" : 37, + "index" : 5, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.100GMT", + "duration" : 1915, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 256, + "executorDeserializeCpuTime" : 60890000, + "executorRunTime" : 1596, + "executorCpuTime" : 507192000, + "resultSize" : 2115, + "jvmGcTime" : 62, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 63, + "gettingResultTime" : 0 + }, + "46" : { + "taskId" : 46, + "index" : 14, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.132GMT", + "duration" : 1905, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 218, + "executorDeserializeCpuTime" : 51464000, + "executorRunTime" : 1618, + "executorCpuTime" : 490927000, + "resultSize" : 2115, + "jvmGcTime" : 66, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 69, + "gettingResultTime" : 0 + }, + "38" : { + "taskId" : 38, + "index" : 6, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.104GMT", + "duration" : 1835, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 255, + "executorDeserializeCpuTime" : 60358000, + "executorRunTime" : 1498, + "executorCpuTime" : 414110000, + "resultSize" : 2158, + "jvmGcTime" : 62, + "resultSerializationTime" : 11, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 71, + "gettingResultTime" : 0 + }, + "33" : { + "taskId" : 33, + "index" : 1, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.082GMT", + "duration" : 1943, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 267, + "executorDeserializeCpuTime" : 54442000, + "executorRunTime" : 1597, + "executorCpuTime" : 519178000, + "resultSize" : 2115, + "jvmGcTime" : 62, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 79, + "gettingResultTime" : 0 + }, + "41" : { + "taskId" : 41, + "index" : 9, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.116GMT", + "duration" : 1916, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 240, + "executorDeserializeCpuTime" : 55787000, + "executorRunTime" : 1614, + "executorCpuTime" : 489923000, + "resultSize" : 2115, + "jvmGcTime" : 66, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 62, + "gettingResultTime" : 0 + }, + "32" : { + "taskId" : 32, + "index" : 0, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.077GMT", + "duration" : 1960, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 271, + "executorDeserializeCpuTime" : 56827000, + "executorRunTime" : 1619, + "executorCpuTime" : 496683000, + "resultSize" : 2115, + "jvmGcTime" : 66, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 70, + "gettingResultTime" : 0 + }, + "34" : { + "taskId" : 34, + "index" : 2, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.087GMT", + "duration" : 1939, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 265, + "executorDeserializeCpuTime" : 69492000, + "executorRunTime" : 1606, + "executorCpuTime" : 508433000, + "resultSize" : 2115, + "jvmGcTime" : 66, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 68, + "gettingResultTime" : 0 + }, + "45" : { + "taskId" : 45, + "index" : 13, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.129GMT", + "duration" : 1895, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 221, + "executorDeserializeCpuTime" : 54222000, + "executorRunTime" : 1595, + "executorCpuTime" : 495138000, + "resultSize" : 2115, + "jvmGcTime" : 62, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 79, + "gettingResultTime" : 0 + }, + "44" : { + "taskId" : 44, + "index" : 12, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.126GMT", + "duration" : 1917, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 222, + "executorDeserializeCpuTime" : 51988000, + "executorRunTime" : 1624, + "executorCpuTime" : 498187000, + "resultSize" : 2115, + "jvmGcTime" : 66, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 71, + "gettingResultTime" : 0 + }, + "39" : { + "taskId" : 39, + "index" : 7, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.109GMT", + "duration" : 1915, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 254, + "executorDeserializeCpuTime" : 64380000, + "executorRunTime" : 1596, + "executorCpuTime" : 539451000, + "resultSize" : 2115, + "jvmGcTime" : 62, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 65, + "gettingResultTime" : 0 + }, + "35" : { + "taskId" : 35, + "index" : 3, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.091GMT", + "duration" : 1925, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 263, + "executorDeserializeCpuTime" : 62944000, + "executorRunTime" : 1598, + "executorCpuTime" : 502908000, + "resultSize" : 2115, + "jvmGcTime" : 62, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 64, + "gettingResultTime" : 0 + }, + "43" : { + "taskId" : 43, + "index" : 11, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.123GMT", + "duration" : 1906, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 225, + "executorDeserializeCpuTime" : 48849000, + "executorRunTime" : 1609, + "executorCpuTime" : 502120000, + "resultSize" : 2115, + "jvmGcTime" : 66, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 72, + "gettingResultTime" : 0 + }, + "40" : { + "taskId" : 40, + "index" : 8, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.112GMT", + "duration" : 1904, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 246, + "executorDeserializeCpuTime" : 69760000, + "executorRunTime" : 1595, + "executorCpuTime" : 510597000, + "resultSize" : 2115, + "jvmGcTime" : 62, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 63, + "gettingResultTime" : 0 + }, + "36" : { + "taskId" : 36, + "index" : 4, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.095GMT", + "duration" : 1920, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 260, + "executorDeserializeCpuTime" : 112849000, + "executorRunTime" : 1596, + "executorCpuTime" : 503010000, + "resultSize" : 2115, + "jvmGcTime" : 62, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 64, + "gettingResultTime" : 0 + }, + "47" : { + "taskId" : 47, + "index" : 15, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.136GMT", + "duration" : 1878, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 213, + "executorDeserializeCpuTime" : 47496000, + "executorRunTime" : 1594, + "executorCpuTime" : 1324251000, + "resultSize" : 2115, + "jvmGcTime" : 52, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 71, + "gettingResultTime" : 0 + } + }, + "executorSummary" : { + "0" : { + "taskTime" : 30596, + "failedTasks" : 0, + "succeededTasks" : 16, + "killedTasks" : 0, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleRead" : 0, + "shuffleReadRecords" : 0, + "shuffleWrite" : 0, + "shuffleWriteRecords" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "isBlacklistedForStage" : false, + "peakMemoryMetrics" : { + "JVMHeapMemory" : 0, + "JVMOffHeapMemory" : 0, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 0, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 0, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 0, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 0, + "ProcessTreeJVMRSSMemory" : 0, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 0, + "MinorGCTime" : 0, + "MajorGCCount" : 0, + "MajorGCTime" : 0 + } + }, + "driver" : { + "taskTime" : 0, + "failedTasks" : 0, + "succeededTasks" : 0, + "killedTasks" : 0, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleRead" : 0, + "shuffleReadRecords" : 0, + "shuffleWrite" : 0, + "shuffleWriteRecords" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "isBlacklistedForStage" : false, + "peakMemoryMetrics" : { + "JVMHeapMemory" : 213367864, + "JVMOffHeapMemory" : 189011656, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 2133349, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 2133349, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 282024, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 0, + "ProcessTreeJVMRSSMemory" : 0, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 13, + "MinorGCTime" : 115, + "MajorGCCount" : 4, + "MajorGCTime" : 339 + } + } + }, + "killedTasksSummary" : { }, + "resourceProfileId" : 0, + "peakExecutorMetrics" : { + "JVMHeapMemory" : 213367864, + "JVMOffHeapMemory" : 189011656, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 2133349, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 2133349, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 282024, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 0, + "ProcessTreeJVMRSSMemory" : 0, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 13, + "MinorGCTime" : 115, + "MajorGCCount" : 4, + "MajorGCTime" : 339 + } +} diff --git a/core/src/test/resources/spark-events/app-20200706201101-0003 b/core/src/test/resources/spark-events/app-20200706201101-0003 new file mode 100644 index 0000000000000..b2923ca0f001f --- /dev/null +++ b/core/src/test/resources/spark-events/app-20200706201101-0003 @@ -0,0 +1,124 @@ +{"Event":"SparkListenerLogStart","Spark Version":"3.1.0-SNAPSHOT"} +{"Event":"SparkListenerResourceProfileAdded","Resource Profile Id":0,"Executor Resource Requests":{"cores":{"Resource Name":"cores","Amount":1,"Discovery Script":"","Vendor":""},"memory":{"Resource Name":"memory","Amount":1024,"Discovery Script":"","Vendor":""}},"Task Resource Requests":{"cpus":{"Resource Name":"cpus","Amount":1.0}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"localhost","Port":64415},"Maximum Memory":384093388,"Timestamp":1594091461118,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre","Java Version":"1.8.0_231 (Oracle Corporation)","Scala Version":"version 2.12.10"},"Spark Properties":{"spark.driver.host":"localhost","spark.eventLog.enabled":"true","spark.driver.port":"64413","spark.repl.class.uri":"spark://localhost:64413/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/p_/5j1mtw1x0579b34vy4ztcbs40000gn/T/spark-0c713088-21d7-4b35-9b52-7a4dd4d15ae1/repl-361cef81-42cd-4875-a8a6-ce9dfe55682a","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.submit.pyFiles":"","spark.ui.showConsoleProgress":"true","spark.executor.id":"driver","spark.eventLog.logStageExecutorMetrics":"true","spark.submit.deployMode":"client","spark.master":"*********(redacted)","spark.home":"*********(redacted)","spark.sql.catalogImplementation":"in-memory","spark.app.id":"app-20200706201101-0003"},"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":"2","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":"/var/folders/p_/5j1mtw1x0579b34vy4ztcbs40000gn/T/","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":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"*********(redacted)","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib","user.dir":"*********(redacted)","java.library.path":"*********(redacted)/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"25.231-b11","jetty.git.hash":"ab228fde9e55e9164c738d7fa121f8ac5acd51c9","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.8.0_231-b11","java.vm.info":"mixed mode","java.ext.dirs":"*********(redacted)/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","scala.usejavacp":"true","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Los_Angeles","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.15.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","jdk.lang.Process.launchMechanism":"POSIX_SPAWN","user.country":"US","sun.jnu.encoding":"UTF-8","user.language":"en","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"terryk","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master spark://*********(redacted) --conf spark.eventLog.logStageExecutorMetrics=true --conf spark.eventLog.enabled=true --class org.apache.spark.repl.Main --name Spark shell spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre","java.version":"1.8.0_231","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"*********(redacted)/audience-annotations-0.5.0.jar":"System Classpath","*********(redacted)/zstd-jni-1.4.5-2.jar":"System Classpath","*********(redacted)/hadoop-yarn-common-3.2.0.jar":"System Classpath","*********(redacted)/apache-spark/common/unsafe/target/scala-2.12/classes/":"System Classpath","*********(redacted)/kerb-identity-1.0.1.jar":"System Classpath","*********(redacted)/jersey-hk2-2.30.jar":"System Classpath","*********(redacted)/apache-spark/mllib/target/jars/*":"System Classpath","*********(redacted)/slf4j-log4j12-1.7.30.jar":"System Classpath","*********(redacted)/kerby-pkix-1.0.1.jar":"System Classpath","*********(redacted)/metrics-jmx-4.1.1.jar":"System Classpath","*********(redacted)/xz-1.5.jar":"System Classpath","*********(redacted)/jakarta.validation-api-2.0.2.jar":"System Classpath","*********(redacted)/jetty-webapp-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/metrics-core-4.1.1.jar":"System Classpath","*********(redacted)/apache-spark/graphx/target/scala-2.12/classes/":"System Classpath","*********(redacted)/metrics-graphite-4.1.1.jar":"System Classpath","*********(redacted)/jersey-server-2.30.jar":"System Classpath","*********(redacted)/jetty-jndi-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/jetty-proxy-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/parquet-jackson-1.10.1.jar":"System Classpath","*********(redacted)/commons-crypto-1.0.0.jar":"System Classpath","*********(redacted)/xbean-asm7-shaded-4.15.jar":"System Classpath","*********(redacted)/spark-kvstore_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/netty-3.10.6.Final.jar":"System Classpath","*********(redacted)/jsr305-3.0.2.jar":"System Classpath","*********(redacted)/osgi-resource-locator-1.0.3.jar":"System Classpath","*********(redacted)/kerb-server-1.0.1.jar":"System Classpath","*********(redacted)/avro-1.8.2.jar":"System Classpath","*********(redacted)/jersey-media-jaxb-2.30.jar":"System Classpath","*********(redacted)/accessors-smart-1.2.jar":"System Classpath","*********(redacted)/spark-network-shuffle_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/protobuf-java-2.5.0.jar":"System Classpath","*********(redacted)/minlog-1.3.0.jar":"System Classpath","*********(redacted)/scala-parser-combinators_2.12-1.1.2.jar":"System Classpath","*********(redacted)/jetty-continuation-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/apache-spark/sql/hive/target/scala-2.12/classes/":"System Classpath","*********(redacted)/commons-net-3.6.jar":"System Classpath","*********(redacted)/apache-spark/core/target/jars/*":"System Classpath","*********(redacted)/istack-commons-runtime-3.0.8.jar":"System Classpath","*********(redacted)/jsp-api-2.1.jar":"System Classpath","*********(redacted)/hadoop-annotations-3.2.0.jar":"System Classpath","*********(redacted)/spark-launcher_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/jetty-servlets-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/hadoop-hdfs-client-3.2.0.jar":"System Classpath","*********(redacted)/hive-storage-api-2.7.1.jar":"System Classpath","*********(redacted)/aopalliance-repackaged-2.6.1.jar":"System Classpath","*********(redacted)/kerby-xdr-1.0.1.jar":"System Classpath","*********(redacted)/orc-mapreduce-1.5.10.jar":"System Classpath","*********(redacted)/jackson-databind-2.10.0.jar":"System Classpath","*********(redacted)/spark-mllib_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/hadoop-mapreduce-client-jobclient-3.2.0.jar":"System Classpath","*********(redacted)/hk2-utils-2.6.1.jar":"System Classpath","*********(redacted)/jetty-security-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/hadoop-client-3.2.0.jar":"System Classpath","*********(redacted)/hadoop-auth-3.2.0.jar":"System Classpath","*********(redacted)/gson-2.2.4.jar":"System Classpath","*********(redacted)/htrace-core4-4.1.0-incubating.jar":"System Classpath","*********(redacted)/curator-client-2.12.0.jar":"System Classpath","*********(redacted)/kerby-util-1.0.1.jar":"System Classpath","*********(redacted)/jackson-jaxrs-base-2.9.5.jar":"System Classpath","*********(redacted)/jackson-module-paranamer-2.10.0.jar":"System Classpath","*********(redacted)/commons-beanutils-1.9.3.jar":"System Classpath","*********(redacted)/woodstox-core-5.0.3.jar":"System Classpath","*********(redacted)/arrow-memory-0.15.1.jar":"System Classpath","*********(redacted)/activation-1.1.1.jar":"System Classpath","*********(redacted)/okhttp-2.7.5.jar":"System Classpath","*********(redacted)/jackson-mapper-asl-1.9.13.jar":"System Classpath","*********(redacted)/core-1.1.2.jar":"System Classpath","*********(redacted)/jetty-plus-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/apache-spark/resource-managers/yarn/target/scala-2.12/classes/":"System Classpath","*********(redacted)/antlr4-runtime-4.7.1.jar":"System Classpath","*********(redacted)/commons-compress-1.8.1.jar":"System Classpath","*********(redacted)/spark-tags_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/orc-shims-1.5.10.jar":"System Classpath","*********(redacted)/opencsv-2.3.jar":"System Classpath","*********(redacted)/compress-lzf-1.0.3.jar":"System Classpath","*********(redacted)/jackson-core-asl-1.9.13.jar":"System Classpath","*********(redacted)/objenesis-2.5.1.jar":"System Classpath","*********(redacted)/apache-spark/conf/":"System Classpath","*********(redacted)/apache-spark/common/network-common/target/scala-2.12/classes/":"System Classpath","*********(redacted)/httpcore-4.4.4.jar":"System Classpath","*********(redacted)/jcip-annotations-1.0-1.jar":"System Classpath","*********(redacted)/apache-spark/launcher/target/scala-2.12/classes/":"System Classpath","*********(redacted)/apache-spark/resource-managers/mesos/target/scala-2.12/classes":"System Classpath","*********(redacted)/apache-spark/sql/hive-thriftserver/target/scala-2.12/classes/":"System Classpath","*********(redacted)/jetty-util-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/jetty-server-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/jetty-xml-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/scala-collection-compat_2.12-2.1.1.jar":"System Classpath","*********(redacted)/javax.servlet-api-3.1.0.jar":"System Classpath","*********(redacted)/jackson-core-2.10.0.jar":"System Classpath","*********(redacted)/jetty-client-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/netty-all-4.1.47.Final.jar":"System Classpath","*********(redacted)/arpack_combined_all-0.1.jar":"System Classpath","*********(redacted)/jersey-container-servlet-core-2.30.jar":"System Classpath","*********(redacted)/paranamer-2.8.jar":"System Classpath","*********(redacted)/spark-sql_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/hadoop-common-3.2.0.jar":"System Classpath","*********(redacted)/jersey-common-2.30.jar":"System Classpath","*********(redacted)/hk2-locator-2.6.1.jar":"System Classpath","*********(redacted)/okio-1.6.0.jar":"System Classpath","*********(redacted)/jetty-http-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/jakarta.xml.bind-api-2.3.2.jar":"System Classpath","*********(redacted)/ivy-2.4.0.jar":"System Classpath","*********(redacted)/apache-spark/mllib/target/scala-2.12/classes/":"System Classpath","*********(redacted)/scala-library-2.12.10.jar":"System Classpath","*********(redacted)/scala-xml_2.12-1.2.0.jar":"System Classpath","*********(redacted)/apache-spark/streaming/target/scala-2.12/classes/":"System Classpath","*********(redacted)/spark-repl_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/apache-spark/common/tags/target/scala-2.12/classes/":"System Classpath","*********(redacted)/kerby-config-1.0.1.jar":"System Classpath","*********(redacted)/zookeeper-3.4.14.jar":"System Classpath","*********(redacted)/janino-3.1.2.jar":"System Classpath","*********(redacted)/breeze_2.12-1.0.jar":"System Classpath","*********(redacted)/hk2-api-2.6.1.jar":"System Classpath","*********(redacted)/hadoop-mapreduce-client-common-3.2.0.jar":"System Classpath","*********(redacted)/flatbuffers-java-1.9.0.jar":"System Classpath","*********(redacted)/curator-framework-2.13.0.jar":"System Classpath","*********(redacted)/metrics-jvm-4.1.1.jar":"System Classpath","*********(redacted)/stax2-api-3.1.4.jar":"System Classpath","*********(redacted)/json-smart-2.3.jar":"System Classpath","*********(redacted)/parquet-hadoop-1.10.1.jar":"System Classpath","*********(redacted)/kerb-common-1.0.1.jar":"System Classpath","*********(redacted)/breeze-macros_2.12-1.0.jar":"System Classpath","*********(redacted)/scala-compiler-2.12.10.jar":"System Classpath","*********(redacted)/guava-14.0.1.jar":"System Classpath","*********(redacted)/jul-to-slf4j-1.7.30.jar":"System Classpath","*********(redacted)/spark-assembly_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/spark-core_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/spark-graphx_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/jakarta.ws.rs-api-2.1.6.jar":"System Classpath","*********(redacted)/spark-unsafe_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/jetty-io-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/JTransforms-3.1.jar":"System Classpath","*********(redacted)/commons-io-2.5.jar":"System Classpath","*********(redacted)/spark-catalyst_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/spire_2.12-0.17.0-M1.jar":"System Classpath","*********(redacted)/commons-cli-1.2.jar":"System Classpath","*********(redacted)/shims-0.7.45.jar":"System Classpath","*********(redacted)/spotbugs-annotations-3.1.9.jar":"System Classpath","*********(redacted)/kerb-simplekdc-1.0.1.jar":"System Classpath","*********(redacted)/commons-codec-1.11.jar":"System Classpath","*********(redacted)/jcl-over-slf4j-1.7.30.jar":"System Classpath","*********(redacted)/lz4-java-1.7.1.jar":"System Classpath","*********(redacted)/aopalliance-1.0.jar":"System Classpath","*********(redacted)/jackson-jaxrs-json-provider-2.9.5.jar":"System Classpath","*********(redacted)/json4s-jackson_2.12-3.6.6.jar":"System Classpath","*********(redacted)/commons-math3-3.4.1.jar":"System Classpath","*********(redacted)/shapeless_2.12-2.3.3.jar":"System Classpath","*********(redacted)/spark-mllib-local_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/dnsjava-2.1.7.jar":"System Classpath","*********(redacted)/unused-1.0.0.jar":"System Classpath","*********(redacted)/log4j-1.2.17.jar":"System Classpath","*********(redacted)/kerb-client-1.0.1.jar":"System Classpath","*********(redacted)/commons-lang-2.6.jar":"System Classpath","*********(redacted)/apache-spark/repl/target/scala-2.12/classes/":"System Classpath","*********(redacted)/kerb-util-1.0.1.jar":"System Classpath","*********(redacted)/slf4j-api-1.7.30.jar":"System Classpath","*********(redacted)/jackson-annotations-2.10.0.jar":"System Classpath","*********(redacted)/kerb-core-1.0.1.jar":"System Classpath","*********(redacted)/apache-spark/sql/catalyst/target/scala-2.12/classes/":"System Classpath","*********(redacted)/spire-macros_2.12-0.17.0-M1.jar":"System Classpath","*********(redacted)/apache-spark/examples/target/scala-2.12/classes/":"System Classpath","*********(redacted)/jakarta.annotation-api-1.3.5.jar":"System Classpath","*********(redacted)/json4s-scalap_2.12-3.6.6.jar":"System Classpath","*********(redacted)/apache-spark/core/target/scala-2.12/classes/":"System Classpath","*********(redacted)/arrow-format-0.15.1.jar":"System Classpath","*********(redacted)/kryo-shaded-4.0.2.jar":"System Classpath","*********(redacted)/scala-reflect-2.12.10.jar":"System Classpath","*********(redacted)/kerb-admin-1.0.1.jar":"System Classpath","*********(redacted)/hadoop-yarn-client-3.2.0.jar":"System Classpath","*********(redacted)/nimbus-jose-jwt-4.41.1.jar":"System Classpath","*********(redacted)/apache-spark/common/kvstore/target/scala-2.12/classes/":"System Classpath","*********(redacted)/oro-2.0.8.jar":"System Classpath","*********(redacted)/jakarta.inject-2.6.1.jar":"System Classpath","*********(redacted)/chill-java-0.9.5.jar":"System Classpath","*********(redacted)/cats-kernel_2.12-2.0.0-M4.jar":"System Classpath","*********(redacted)/jersey-container-servlet-2.30.jar":"System Classpath","*********(redacted)/py4j-0.10.9.jar":"System Classpath","*********(redacted)/parquet-format-2.4.0.jar":"System Classpath","*********(redacted)/apache-spark/sql/core/target/scala-2.12/classes/":"System Classpath","*********(redacted)/jline-2.14.6.jar":"System Classpath","*********(redacted)/JLargeArrays-1.5.jar":"System Classpath","*********(redacted)/kerby-asn1-1.0.1.jar":"System Classpath","*********(redacted)/jaxb-runtime-2.3.2.jar":"System Classpath","*********(redacted)/pmml-model-1.4.8.jar":"System Classpath","*********(redacted)/parquet-encoding-1.10.1.jar":"System Classpath","*********(redacted)/machinist_2.12-0.6.8.jar":"System Classpath","*********(redacted)/commons-compiler-3.1.2.jar":"System Classpath","*********(redacted)/kerb-crypto-1.0.1.jar":"System Classpath","*********(redacted)/aircompressor-0.10.jar":"System Classpath","*********(redacted)/leveldbjni-all-1.8.jar":"System Classpath","*********(redacted)/metrics-json-4.1.1.jar":"System Classpath","*********(redacted)/jackson-module-jaxb-annotations-2.9.5.jar":"System Classpath","*********(redacted)/jaxb-api-2.2.11.jar":"System Classpath","*********(redacted)/spire-util_2.12-0.17.0-M1.jar":"System Classpath","*********(redacted)/jetty-servlet-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/spark-network-common_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/spark-streaming_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/json4s-core_2.12-3.6.6.jar":"System Classpath","*********(redacted)/stream-2.9.6.jar":"System Classpath","*********(redacted)/spark-sketch_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/chill_2.12-0.9.5.jar":"System Classpath","*********(redacted)/commons-configuration2-2.1.1.jar":"System Classpath","*********(redacted)/univocity-parsers-2.8.3.jar":"System Classpath","*********(redacted)/parquet-common-1.10.1.jar":"System Classpath","*********(redacted)/threeten-extra-1.5.0.jar":"System Classpath","*********(redacted)/token-provider-1.0.1.jar":"System Classpath","*********(redacted)/commons-text-1.6.jar":"System Classpath","*********(redacted)/apache-spark/common/sketch/target/scala-2.12/classes/":"System Classpath","*********(redacted)/javax.inject-1.jar":"System Classpath","*********(redacted)/httpclient-4.5.2.jar":"System Classpath","*********(redacted)/avro-ipc-1.8.2.jar":"System Classpath","*********(redacted)/curator-recipes-2.13.0.jar":"System Classpath","*********(redacted)/commons-collections-3.2.2.jar":"System Classpath","*********(redacted)/jersey-client-2.30.jar":"System Classpath","*********(redacted)/snappy-java-1.1.7.5.jar":"System Classpath","*********(redacted)/algebra_2.12-2.0.0-M2.jar":"System Classpath","*********(redacted)/javassist-3.25.0-GA.jar":"System Classpath","*********(redacted)/arrow-vector-0.15.1.jar":"System Classpath","*********(redacted)/avro-mapred-1.8.2-hadoop2.jar":"System Classpath","*********(redacted)/re2j-1.1.jar":"System Classpath","*********(redacted)/parquet-column-1.10.1.jar":"System Classpath","*********(redacted)/pyrolite-4.30.jar":"System Classpath","*********(redacted)/apache-spark/common/network-shuffle/target/scala-2.12/classes/":"System Classpath","*********(redacted)/apache-spark/common/network-yarn/target/scala-2.12/classes":"System Classpath","*********(redacted)/hadoop-yarn-api-3.2.0.jar":"System Classpath","*********(redacted)/macro-compat_2.12-1.1.1.jar":"System Classpath","*********(redacted)/spire-platform_2.12-0.17.0-M1.jar":"System Classpath","*********(redacted)/commons-lang3-3.10.jar":"System Classpath","*********(redacted)/json4s-ast_2.12-3.6.6.jar":"System Classpath","*********(redacted)/RoaringBitmap-0.7.45.jar":"System Classpath","*********(redacted)/orc-core-1.5.10.jar":"System Classpath","*********(redacted)/jackson-module-scala_2.12-2.10.0.jar":"System Classpath","*********(redacted)/hadoop-mapreduce-client-core-3.2.0.jar":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"app-20200706201101-0003","Timestamp":1594091460235,"User":"terryk"} +{"Event":"SparkListenerExecutorAdded","Timestamp":1594091463318,"Executor ID":"0","Executor Info":{"Host":"127.0.0.1","Total Cores":16,"Log Urls":{"stdout":"http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout","stderr":"http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr"},"Attributes":{},"Resources":{},"Resource Profile Id":0}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"0","Host":"127.0.0.1","Port":64419},"Maximum Memory":384093388,"Timestamp":1594091463413,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":0,"description":"foreach at :26","details":"org.apache.spark.sql.Dataset.foreach(Dataset.scala:2862)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:26)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:30)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:32)\n$line19.$read$$iw$$iw$$iw$$iw$$iw.(:34)\n$line19.$read$$iw$$iw$$iw$$iw.(:36)\n$line19.$read$$iw$$iw$$iw.(:38)\n$line19.$read$$iw$$iw.(:40)\n$line19.$read$$iw.(:42)\n$line19.$read.(:44)\n$line19.$read$.(:48)\n$line19.$read$.()\n$line19.$eval$.$print$lzycompute(:7)\n$line19.$eval$.$print(:6)\n$line19.$eval.$print()\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)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:745)","physicalPlanDescription":"== Physical Plan ==\nDeserializeToObject (8)\n+- * BroadcastHashJoin Inner BuildLeft (7)\n :- BroadcastExchange (5)\n : +- * BroadcastHashJoin Inner BuildRight (4)\n : :- * LocalTableScan (1)\n : +- BroadcastExchange (3)\n : +- LocalTableScan (2)\n +- * LocalTableScan (6)\n\n\n(1) LocalTableScan [codegen id : 1]\nOutput [3]: [i1#10, j1#11, k1#12]\nArguments: [i1#10, j1#11, k1#12]\n\n(2) LocalTableScan\nOutput [3]: [i2#26, j2#27, k2#28]\nArguments: [i2#26, j2#27, k2#28]\n\n(3) BroadcastExchange\nInput [3]: [i2#26, j2#27, k2#28]\nArguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint))), [id=#23]\n\n(4) BroadcastHashJoin [codegen id : 1]\nLeft keys [1]: [i1#10]\nRight keys [1]: [i2#26]\nJoin condition: None\n\n(5) BroadcastExchange\nInput [6]: [i1#10, j1#11, k1#12, i2#26, j2#27, k2#28]\nArguments: HashedRelationBroadcastMode(List(cast(input[3, int, false] as bigint))), [id=#32]\n\n(6) LocalTableScan\nOutput [3]: [i3#42, j3#43, k3#44]\nArguments: [i3#42, j3#43, k3#44]\n\n(7) BroadcastHashJoin [codegen id : 2]\nLeft keys [1]: [i2#26]\nRight keys [1]: [i3#42]\nJoin condition: None\n\n(8) DeserializeToObject\nInput [9]: [i1#10, j1#11, k1#12, i2#26, j2#27, k2#28, i3#42, j3#43, k3#44]\nArguments: createexternalrow(i1#10, j1#11, k1#12.toString, i2#26, j2#27, k2#28.toString, i3#42, j3#43, k3#44.toString, StructField(i1,IntegerType,false), StructField(j1,IntegerType,false), StructField(k1,StringType,true), StructField(i2,IntegerType,false), StructField(j2,IntegerType,false), StructField(k2,StringType,true), StructField(i3,IntegerType,false), StructField(j3,IntegerType,false), StructField(k3,StringType,true)), obj#93: org.apache.spark.sql.Row\n\n","sparkPlanInfo":{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject createexternalrow(i1#10, j1#11, k1#12.toString, i2#26, j2#27, k2#28.toString, i3#42, j3#43, k3#44.toString, StructField(i1,IntegerType,false), StructField(j1,IntegerType,false), StructField(k1,StringType,true), StructField(i2,IntegerType,false), StructField(j2,IntegerType,false), StructField(k2,StringType,true), StructField(i3,IntegerType,false), StructField(j3,IntegerType,false), StructField(k3,StringType,true)), obj#93: org.apache.spark.sql.Row","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"BroadcastHashJoin","simpleString":"BroadcastHashJoin [i2#26], [i3#42], Inner, BuildLeft","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"BroadcastExchange","simpleString":"BroadcastExchange HashedRelationBroadcastMode(List(cast(input[3, int, false] as bigint))), [id=#32]","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"BroadcastHashJoin","simpleString":"BroadcastHashJoin [i1#10], [i2#26], Inner, BuildRight","children":[{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [i1#10, j1#11, k1#12]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":8,"metricType":"sum"}]},{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"BroadcastExchange","simpleString":"BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint))), [id=#23]","children":[{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [i2#26, j2#27, k2#28]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":13,"metricType":"sum"}]}],"metadata":{},"metrics":[{"name":"data size","accumulatorId":9,"metricType":"size"},{"name":"time to collect","accumulatorId":10,"metricType":"timing"},{"name":"time to build","accumulatorId":11,"metricType":"timing"},{"name":"time to broadcast","accumulatorId":12,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":7,"metricType":"sum"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":6,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"data size","accumulatorId":2,"metricType":"size"},{"name":"time to collect","accumulatorId":3,"metricType":"timing"},{"name":"time to build","accumulatorId":4,"metricType":"timing"},{"name":"time to broadcast","accumulatorId":5,"metricType":"timing"}]}],"metadata":{},"metrics":[]},{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [i3#42, j3#43, k3#44]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":14,"metricType":"sum"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1,"metricType":"sum"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":0,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1594091478577} +{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1594091478844,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Number of Tasks":16,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"3\",\"name\":\"LocalTableScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"3\",\"name\":\"LocalTableScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:185)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[0],"Properties":{"spark.driver.host":"localhost","spark.eventLog.enabled":"true","spark.job.interruptOnCancel":"true","spark.driver.port":"64413","spark.repl.class.uri":"spark://localhost:64413/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/p_/5j1mtw1x0579b34vy4ztcbs40000gn/T/spark-0c713088-21d7-4b35-9b52-7a4dd4d15ae1/repl-361cef81-42cd-4875-a8a6-ce9dfe55682a","spark.app.name":"Spark shell","spark.rdd.scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.ui.showConsoleProgress":"true","spark.job.description":"broadcast exchange (runId bb0234e5-4157-49a4-b40c-6d538d9f2ec8)","spark.executor.id":"driver","spark.jobGroup.id":"bb0234e5-4157-49a4-b40c-6d538d9f2ec8","spark.eventLog.logStageExecutorMetrics":"true","spark.submit.deployMode":"client","spark.master":"*********(redacted)","spark.home":"*********(redacted)","spark.sql.execution.id":"0","spark.sql.catalogImplementation":"in-memory","spark.app.id":"app-20200706201101-0003"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Number of Tasks":16,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"3\",\"name\":\"LocalTableScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"3\",\"name\":\"LocalTableScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:185)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)","Submission Time":1594091478860,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.driver.host":"localhost","spark.eventLog.enabled":"true","spark.job.interruptOnCancel":"true","spark.driver.port":"64413","spark.repl.class.uri":"spark://localhost:64413/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/p_/5j1mtw1x0579b34vy4ztcbs40000gn/T/spark-0c713088-21d7-4b35-9b52-7a4dd4d15ae1/repl-361cef81-42cd-4875-a8a6-ce9dfe55682a","spark.app.name":"Spark shell","spark.rdd.scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.ui.showConsoleProgress":"true","spark.job.description":"broadcast exchange (runId bb0234e5-4157-49a4-b40c-6d538d9f2ec8)","spark.executor.id":"driver","spark.jobGroup.id":"bb0234e5-4157-49a4-b40c-6d538d9f2ec8","spark.eventLog.logStageExecutorMetrics":"true","spark.submit.deployMode":"client","spark.master":"*********(redacted)","spark.home":"*********(redacted)","spark.sql.execution.id":"0","spark.sql.catalogImplementation":"in-memory","spark.app.id":"app-20200706201101-0003"}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1594091479253,"Executor ID":"0","Host":"127.0.0.1","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":1594091479269,"Executor ID":"0","Host":"127.0.0.1","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":1594091479269,"Executor ID":"0","Host":"127.0.0.1","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":3,"Index":3,"Attempt":0,"Launch Time":1594091479270,"Executor ID":"0","Host":"127.0.0.1","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":4,"Index":4,"Attempt":0,"Launch Time":1594091479270,"Executor ID":"0","Host":"127.0.0.1","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":5,"Index":5,"Attempt":0,"Launch Time":1594091479270,"Executor ID":"0","Host":"127.0.0.1","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":6,"Index":6,"Attempt":0,"Launch Time":1594091479271,"Executor ID":"0","Host":"127.0.0.1","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":7,"Index":7,"Attempt":0,"Launch Time":1594091479272,"Executor ID":"0","Host":"127.0.0.1","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":8,"Index":8,"Attempt":0,"Launch Time":1594091479272,"Executor ID":"0","Host":"127.0.0.1","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":9,"Index":9,"Attempt":0,"Launch Time":1594091479273,"Executor ID":"0","Host":"127.0.0.1","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":10,"Index":10,"Attempt":0,"Launch Time":1594091479273,"Executor ID":"0","Host":"127.0.0.1","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":11,"Index":11,"Attempt":0,"Launch Time":1594091479273,"Executor ID":"0","Host":"127.0.0.1","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":12,"Index":12,"Attempt":0,"Launch Time":1594091479274,"Executor ID":"0","Host":"127.0.0.1","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":13,"Index":13,"Attempt":0,"Launch Time":1594091479274,"Executor ID":"0","Host":"127.0.0.1","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":14,"Index":14,"Attempt":0,"Launch Time":1594091479274,"Executor ID":"0","Host":"127.0.0.1","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":15,"Index":15,"Attempt":0,"Launch Time":1594091479274,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1594091479271,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480364,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":109,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1387,"Value":1387,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":8922000,"Value":8922000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":80,"Value":80,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":235295000,"Value":235295000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":962,"Value":962,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":962,"Executor Deserialize CPU Time":235295000,"Executor Run Time":80,"Executor CPU Time":8922000,"Peak Execution Memory":0,"Result Size":1387,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1594091479272,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480367,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":218,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1391,"Value":2778,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":8860000,"Value":17782000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":81,"Value":161,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":213308000,"Value":448603000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":963,"Value":1925,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":963,"Executor Deserialize CPU Time":213308000,"Executor Run Time":81,"Executor CPU Time":8860000,"Peak Execution Memory":0,"Result Size":1391,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1594091479269,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480367,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":327,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1387,"Value":4165,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":10463000,"Value":28245000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":79,"Value":240,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":228677000,"Value":677280000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":962,"Value":2887,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":962,"Executor Deserialize CPU Time":228677000,"Executor Run Time":79,"Executor CPU Time":10463000,"Peak Execution Memory":0,"Result Size":1387,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":13,"Attempt":0,"Launch Time":1594091479274,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480367,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"4","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":436,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1390,"Value":5555,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":8782000,"Value":37027000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":83,"Value":323,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":196368000,"Value":873648000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":962,"Value":3849,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":962,"Executor Deserialize CPU Time":196368000,"Executor Run Time":83,"Executor CPU Time":8782000,"Peak Execution Memory":0,"Result Size":1390,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1594091479253,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480368,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":545,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1384,"Value":6939,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":10820000,"Value":47847000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":77,"Value":400,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":221708000,"Value":1095356000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":962,"Value":4811,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":962,"Executor Deserialize CPU Time":221708000,"Executor Run Time":77,"Executor CPU Time":10820000,"Peak Execution Memory":0,"Result Size":1384,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":11,"Attempt":0,"Launch Time":1594091479273,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480368,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"2","Value":"7","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":654,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1406,"Value":8345,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":13213000,"Value":61060000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":78,"Value":478,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":260380000,"Value":1355736000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":963,"Value":5774,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":963,"Executor Deserialize CPU Time":260380000,"Executor Run Time":78,"Executor CPU Time":13213000,"Peak Execution Memory":0,"Result Size":1406,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":10,"Attempt":0,"Launch Time":1594091479273,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480368,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"8","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":763,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1391,"Value":9736,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":9913000,"Value":70973000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":76,"Value":554,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":210788000,"Value":1566524000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":962,"Value":6736,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":962,"Executor Deserialize CPU Time":210788000,"Executor Run Time":76,"Executor CPU Time":9913000,"Peak Execution Memory":0,"Result Size":1391,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":14,"Attempt":0,"Launch Time":1594091479274,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480368,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"9","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":872,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1391,"Value":11127,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":8784000,"Value":79757000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":79,"Value":633,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":235620000,"Value":1802144000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":964,"Value":7700,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":964,"Executor Deserialize CPU Time":235620000,"Executor Run Time":79,"Executor CPU Time":8784000,"Peak Execution Memory":0,"Result Size":1391,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":12,"Attempt":0,"Launch Time":1594091479274,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480369,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":981,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1391,"Value":12518,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":12053000,"Value":91810000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":86,"Value":719,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":215398000,"Value":2017542000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":963,"Value":8663,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":963,"Executor Deserialize CPU Time":215398000,"Executor Run Time":86,"Executor CPU Time":12053000,"Peak Execution Memory":0,"Result Size":1391,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1594091479270,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480369,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"2","Value":"12","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":1090,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1403,"Value":13921,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":9030000,"Value":100840000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":79,"Value":798,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":196266000,"Value":2213808000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":962,"Value":9625,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":962,"Executor Deserialize CPU Time":196266000,"Executor Run Time":79,"Executor CPU Time":9030000,"Peak Execution Memory":0,"Result Size":1403,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1594091479273,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480369,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"13","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":1199,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1390,"Value":15311,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":10087000,"Value":110927000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":79,"Value":877,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":195342000,"Value":2409150000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":962,"Value":10587,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":962,"Executor Deserialize CPU Time":195342000,"Executor Run Time":79,"Executor CPU Time":10087000,"Peak Execution Memory":0,"Result Size":1390,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":15,"Attempt":0,"Launch Time":1594091479274,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480369,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"2","Value":"15","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":1308,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1411,"Value":16722,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":12920000,"Value":123847000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":86,"Value":963,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":236044000,"Value":2645194000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":961,"Value":11548,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":961,"Executor Deserialize CPU Time":236044000,"Executor Run Time":86,"Executor CPU Time":12920000,"Peak Execution Memory":0,"Result Size":1411,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1594091479270,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480370,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"16","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":21,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":1417,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1430,"Value":18152,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":10478000,"Value":134325000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":77,"Value":1040,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":205925000,"Value":2851119000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":962,"Value":12510,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":962,"Executor Deserialize CPU Time":205925000,"Executor Run Time":77,"Executor CPU Time":10478000,"Peak Execution Memory":0,"Result Size":1430,"JVM GC Time":109,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1594091479270,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480370,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"17","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":1526,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1387,"Value":19539,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":8972000,"Value":143297000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":87,"Value":1127,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":206247000,"Value":3057366000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":963,"Value":13473,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":963,"Executor Deserialize CPU Time":206247000,"Executor Run Time":87,"Executor CPU Time":8972000,"Peak Execution Memory":0,"Result Size":1387,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1594091479272,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480370,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"2","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":21,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":1635,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1452,"Value":20991,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":20898000,"Value":164195000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":83,"Value":1210,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":389356000,"Value":3446722000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":963,"Value":14436,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":963,"Executor Deserialize CPU Time":389356000,"Executor Run Time":83,"Executor CPU Time":20898000,"Peak Execution Memory":0,"Result Size":1452,"JVM GC Time":109,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1594091479269,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480370,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"20","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":1744,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1384,"Value":22375,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":8668000,"Value":172863000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":82,"Value":1292,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":222167000,"Value":3668889000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":963,"Value":15399,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":963,"Executor Deserialize CPU Time":222167000,"Executor Run Time":82,"Executor CPU Time":8668000,"Peak Execution Memory":0,"Result Size":1384,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"0","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Number of Tasks":16,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"3\",\"name\":\"LocalTableScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"3\",\"name\":\"LocalTableScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:185)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)","Submission Time":1594091478860,"Completion Time":1594091480381,"Accumulables":[{"ID":17,"Name":"internal.metrics.executorRunTime","Value":1292,"Internal":true,"Count Failed Values":true},{"ID":20,"Name":"internal.metrics.jvmGCTime","Value":1744,"Internal":true,"Count Failed Values":true},{"ID":13,"Name":"number of output rows","Value":"20","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Value":3668889000,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Value":22375,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Value":172863000,"Internal":true,"Count Failed Values":true},{"ID":21,"Name":"internal.metrics.resultSerializationTime","Value":3,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Value":15399,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1594091480385,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":0,"accumUpdates":[[9,1048632],[10,1736],[11,37],[12,5]]} +{"Event":"SparkListenerJobStart","Job ID":1,"Submission Time":1594091480498,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Number of Tasks":16,"RDD Info":[{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:185)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[1],"Properties":{"spark.driver.host":"localhost","spark.eventLog.enabled":"true","spark.job.interruptOnCancel":"true","spark.driver.port":"64413","spark.repl.class.uri":"spark://localhost:64413/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/p_/5j1mtw1x0579b34vy4ztcbs40000gn/T/spark-0c713088-21d7-4b35-9b52-7a4dd4d15ae1/repl-361cef81-42cd-4875-a8a6-ce9dfe55682a","spark.app.name":"Spark shell","spark.rdd.scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.ui.showConsoleProgress":"true","spark.job.description":"broadcast exchange (runId bac7289a-c1d8-4966-a6a3-d9f347e13a5d)","spark.executor.id":"driver","spark.jobGroup.id":"bac7289a-c1d8-4966-a6a3-d9f347e13a5d","spark.eventLog.logStageExecutorMetrics":"true","spark.submit.deployMode":"client","spark.master":"*********(redacted)","spark.home":"*********(redacted)","spark.sql.execution.id":"0","spark.sql.catalogImplementation":"in-memory","spark.app.id":"app-20200706201101-0003"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Number of Tasks":16,"RDD Info":[{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:185)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)","Submission Time":1594091480499,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.driver.host":"localhost","spark.eventLog.enabled":"true","spark.job.interruptOnCancel":"true","spark.driver.port":"64413","spark.repl.class.uri":"spark://localhost:64413/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/p_/5j1mtw1x0579b34vy4ztcbs40000gn/T/spark-0c713088-21d7-4b35-9b52-7a4dd4d15ae1/repl-361cef81-42cd-4875-a8a6-ce9dfe55682a","spark.app.name":"Spark shell","spark.rdd.scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.ui.showConsoleProgress":"true","spark.job.description":"broadcast exchange (runId bac7289a-c1d8-4966-a6a3-d9f347e13a5d)","spark.executor.id":"driver","spark.jobGroup.id":"bac7289a-c1d8-4966-a6a3-d9f347e13a5d","spark.eventLog.logStageExecutorMetrics":"true","spark.submit.deployMode":"client","spark.master":"*********(redacted)","spark.home":"*********(redacted)","spark.sql.execution.id":"0","spark.sql.catalogImplementation":"in-memory","spark.app.id":"app-20200706201101-0003"}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1594091480502,"Executor ID":"0","Host":"127.0.0.1","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":17,"Index":1,"Attempt":0,"Launch Time":1594091480503,"Executor ID":"0","Host":"127.0.0.1","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":18,"Index":2,"Attempt":0,"Launch Time":1594091480503,"Executor ID":"0","Host":"127.0.0.1","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":19,"Index":3,"Attempt":0,"Launch Time":1594091480503,"Executor ID":"0","Host":"127.0.0.1","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":20,"Index":4,"Attempt":0,"Launch Time":1594091480503,"Executor ID":"0","Host":"127.0.0.1","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":21,"Index":5,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","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":22,"Index":6,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","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":23,"Index":7,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","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":24,"Index":8,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","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":25,"Index":9,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","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":26,"Index":10,"Attempt":0,"Launch Time":1594091480505,"Executor ID":"0","Host":"127.0.0.1","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":27,"Index":11,"Attempt":0,"Launch Time":1594091480505,"Executor ID":"0","Host":"127.0.0.1","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":28,"Index":12,"Attempt":0,"Launch Time":1594091480506,"Executor ID":"0","Host":"127.0.0.1","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":29,"Index":13,"Attempt":0,"Launch Time":1594091480506,"Executor ID":"0","Host":"127.0.0.1","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":30,"Index":14,"Attempt":0,"Launch Time":1594091480506,"Executor ID":"0","Host":"127.0.0.1","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":31,"Index":15,"Attempt":0,"Launch Time":1594091480507,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":4,"Attempt":0,"Launch Time":1594091480503,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480921,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"20","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"18","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"6","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":1016,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1966,"Value":1966,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":3116000,"Value":3116000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":378,"Value":378,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4258000,"Value":4258000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":29,"Value":29,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":29,"Executor Deserialize CPU Time":4258000,"Executor Run Time":378,"Executor CPU Time":3116000,"Peak Execution Memory":1016,"Result Size":1966,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":1,"Attempt":0,"Launch Time":1594091480503,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480921,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"40","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"36","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"12","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":2032,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1951,"Value":3917,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":2895000,"Value":6011000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":379,"Value":757,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2641000,"Value":6899000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":29,"Value":58,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":29,"Executor Deserialize CPU Time":2641000,"Executor Run Time":379,"Executor CPU Time":2895000,"Peak Execution Memory":1016,"Result Size":1951,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":5,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480923,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"60","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"54","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"18","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":3048,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1957,"Value":5874,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4164000,"Value":10175000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":380,"Value":1137,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2012000,"Value":8911000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":28,"Value":86,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":28,"Executor Deserialize CPU Time":2012000,"Executor Run Time":380,"Executor CPU Time":4164000,"Peak Execution Memory":1016,"Result Size":1957,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":9,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480923,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"80","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"72","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"24","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":4064,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1957,"Value":7831,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4285000,"Value":14460000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":380,"Value":1517,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2749000,"Value":11660000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":26,"Value":112,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":26,"Executor Deserialize CPU Time":2749000,"Executor Run Time":380,"Executor CPU Time":4285000,"Peak Execution Memory":1016,"Result Size":1957,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":7,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480924,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"100","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"21","Value":"93","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"7","Value":"31","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":5080,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2007,"Value":9838,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":16921000,"Value":31381000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":383,"Value":1900,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3854000,"Value":15514000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":26,"Value":138,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":26,"Executor Deserialize CPU Time":3854000,"Executor Run Time":383,"Executor CPU Time":16921000,"Peak Execution Memory":1016,"Result Size":2007,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":14,"Attempt":0,"Launch Time":1594091480506,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480925,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"120","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"111","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"37","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":6096,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1955,"Value":11793,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4798000,"Value":36179000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":382,"Value":2282,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2959000,"Value":18473000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":24,"Value":162,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":24,"Executor Deserialize CPU Time":2959000,"Executor Run Time":382,"Executor CPU Time":4798000,"Peak Execution Memory":1016,"Result Size":1955,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":11,"Attempt":0,"Launch Time":1594091480505,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480926,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"140","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"21","Value":"132","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"7","Value":"44","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":7112,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2015,"Value":13808,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4850000,"Value":41029000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":382,"Value":2664,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4278000,"Value":22751000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":25,"Value":187,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":25,"Executor Deserialize CPU Time":4278000,"Executor Run Time":382,"Executor CPU Time":4850000,"Peak Execution Memory":1016,"Result Size":2015,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":3,"Attempt":0,"Launch Time":1594091480503,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480927,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"21","Value":"161","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"21","Value":"153","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"7","Value":"51","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":8128,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1998,"Value":15806,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4100000,"Value":45129000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":383,"Value":3047,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3622000,"Value":26373000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":28,"Value":215,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":28,"Executor Deserialize CPU Time":3622000,"Executor Run Time":383,"Executor CPU Time":4100000,"Peak Execution Memory":1016,"Result Size":1998,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":10,"Attempt":0,"Launch Time":1594091480505,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480927,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"21","Value":"182","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"171","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"57","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":9144,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1973,"Value":17779,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4350000,"Value":49479000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":383,"Value":3430,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3471000,"Value":29844000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":25,"Value":240,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":25,"Executor Deserialize CPU Time":3471000,"Executor Run Time":383,"Executor CPU Time":4350000,"Peak Execution Memory":1016,"Result Size":1973,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":12,"Attempt":0,"Launch Time":1594091480506,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480927,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"202","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"189","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"63","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":10160,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1966,"Value":19745,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4682000,"Value":54161000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":384,"Value":3814,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2652000,"Value":32496000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":23,"Value":263,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":23,"Executor Deserialize CPU Time":2652000,"Executor Run Time":384,"Executor CPU Time":4682000,"Peak Execution Memory":1016,"Result Size":1966,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":31,"Index":15,"Attempt":0,"Launch Time":1594091480507,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480928,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"21","Value":"223","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"21","Value":"210","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"7","Value":"70","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":11176,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2008,"Value":21753,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":3954000,"Value":58115000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":382,"Value":4196,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4289000,"Value":36785000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":25,"Value":288,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":25,"Executor Deserialize CPU Time":4289000,"Executor Run Time":382,"Executor CPU Time":3954000,"Peak Execution Memory":1016,"Result Size":2008,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":2,"Attempt":0,"Launch Time":1594091480503,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480928,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"243","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"228","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"76","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":12192,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1978,"Value":23731,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":349926000,"Value":408041000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":385,"Value":4581,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":14543000,"Value":51328000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":27,"Value":315,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":27,"Executor Deserialize CPU Time":14543000,"Executor Run Time":385,"Executor CPU Time":349926000,"Peak Execution Memory":1016,"Result Size":1978,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":6,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480928,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"263","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"246","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"82","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":13208,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1965,"Value":25696,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4473000,"Value":412514000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":381,"Value":4962,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3906000,"Value":55234000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":27,"Value":342,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":27,"Executor Deserialize CPU Time":3906000,"Executor Run Time":381,"Executor CPU Time":4473000,"Peak Execution Memory":1016,"Result Size":1965,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":13,"Attempt":0,"Launch Time":1594091480506,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480929,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"283","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"264","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"88","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":14224,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1955,"Value":27651,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":6459000,"Value":418973000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":381,"Value":5343,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3298000,"Value":58532000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":24,"Value":366,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":24,"Executor Deserialize CPU Time":3298000,"Executor Run Time":381,"Executor CPU Time":6459000,"Peak Execution Memory":1016,"Result Size":1955,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":8,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480929,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"21","Value":"304","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"282","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"94","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":15240,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1954,"Value":29605,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4365000,"Value":423338000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":380,"Value":5723,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2411000,"Value":60943000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":28,"Value":394,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":28,"Executor Deserialize CPU Time":2411000,"Executor Run Time":380,"Executor CPU Time":4365000,"Peak Execution Memory":1016,"Result Size":1954,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1594091480502,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480930,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"324","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"300","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"100","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":16256,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1941,"Value":31546,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":3111000,"Value":426449000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":382,"Value":6105,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2723000,"Value":63666000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":30,"Value":424,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":30,"Executor Deserialize CPU Time":2723000,"Executor Run Time":382,"Executor CPU Time":3111000,"Peak Execution Memory":1016,"Result Size":1941,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"0","Stage ID":1,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Number of Tasks":16,"RDD Info":[{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:185)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)","Submission Time":1594091480499,"Completion Time":1594091480930,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Value":63666000,"Internal":true,"Count Failed Values":true},{"ID":8,"Name":"number of output rows","Value":"100","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":44,"Name":"internal.metrics.resultSize","Value":31546,"Internal":true,"Count Failed Values":true},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Value":16256,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Value":424,"Internal":true,"Count Failed Values":true},{"ID":7,"Name":"number of output rows","Value":"300","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":43,"Name":"internal.metrics.executorCpuTime","Value":426449000,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"duration","Value":"324","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":42,"Name":"internal.metrics.executorRunTime","Value":6105,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":1,"Completion Time":1594091480930,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":0,"accumUpdates":[[2,1048616],[3,2276],[4,13],[5,2]]} +{"Event":"SparkListenerJobStart","Job ID":2,"Submission Time":1594091481039,"Stage Infos":[{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"foreach at :26","Number of Tasks":16,"RDD Info":[{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"mapPartitions\"}","Callsite":"foreach at :26","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"foreach at :26","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"1\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"foreach at :26","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"foreach at :26","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"SQLExecutionRDD","Callsite":"foreach at :26","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.foreach(Dataset.scala:2862)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:26)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:30)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:32)\n$line19.$read$$iw$$iw$$iw$$iw$$iw.(:34)\n$line19.$read$$iw$$iw$$iw$$iw.(:36)\n$line19.$read$$iw$$iw$$iw.(:38)\n$line19.$read$$iw$$iw.(:40)\n$line19.$read$$iw.(:42)\n$line19.$read.(:44)\n$line19.$read$.(:48)\n$line19.$read$.()\n$line19.$eval$.$print$lzycompute(:7)\n$line19.$eval$.$print(:6)\n$line19.$eval.$print()\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)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:745)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[2],"Properties":{"spark.driver.host":"localhost","spark.eventLog.enabled":"true","spark.driver.port":"64413","spark.repl.class.uri":"spark://localhost:64413/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/p_/5j1mtw1x0579b34vy4ztcbs40000gn/T/spark-0c713088-21d7-4b35-9b52-7a4dd4d15ae1/repl-361cef81-42cd-4875-a8a6-ce9dfe55682a","spark.app.name":"Spark shell","spark.rdd.scope":"{\"id\":\"19\",\"name\":\"foreach\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.ui.showConsoleProgress":"true","spark.executor.id":"driver","spark.eventLog.logStageExecutorMetrics":"true","spark.submit.deployMode":"client","spark.master":"*********(redacted)","spark.home":"*********(redacted)","spark.sql.execution.id":"0","spark.sql.catalogImplementation":"in-memory","spark.app.id":"app-20200706201101-0003"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"foreach at :26","Number of Tasks":16,"RDD Info":[{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"mapPartitions\"}","Callsite":"foreach at :26","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"foreach at :26","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"1\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"foreach at :26","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"foreach at :26","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"SQLExecutionRDD","Callsite":"foreach at :26","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.foreach(Dataset.scala:2862)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:26)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:30)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:32)\n$line19.$read$$iw$$iw$$iw$$iw$$iw.(:34)\n$line19.$read$$iw$$iw$$iw$$iw.(:36)\n$line19.$read$$iw$$iw$$iw.(:38)\n$line19.$read$$iw$$iw.(:40)\n$line19.$read$$iw.(:42)\n$line19.$read.(:44)\n$line19.$read$.(:48)\n$line19.$read$.()\n$line19.$eval$.$print$lzycompute(:7)\n$line19.$eval$.$print(:6)\n$line19.$eval.$print()\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)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:745)","Submission Time":1594091481040,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.driver.host":"localhost","spark.eventLog.enabled":"true","spark.driver.port":"64413","spark.repl.class.uri":"spark://localhost:64413/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/p_/5j1mtw1x0579b34vy4ztcbs40000gn/T/spark-0c713088-21d7-4b35-9b52-7a4dd4d15ae1/repl-361cef81-42cd-4875-a8a6-ce9dfe55682a","spark.app.name":"Spark shell","spark.rdd.scope":"{\"id\":\"19\",\"name\":\"foreach\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.ui.showConsoleProgress":"true","spark.executor.id":"driver","spark.eventLog.logStageExecutorMetrics":"true","spark.submit.deployMode":"client","spark.master":"*********(redacted)","spark.home":"*********(redacted)","spark.sql.execution.id":"0","spark.sql.catalogImplementation":"in-memory","spark.app.id":"app-20200706201101-0003"}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":32,"Index":0,"Attempt":0,"Launch Time":1594091481077,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":33,"Index":1,"Attempt":0,"Launch Time":1594091481082,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":34,"Index":2,"Attempt":0,"Launch Time":1594091481087,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":35,"Index":3,"Attempt":0,"Launch Time":1594091481091,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":36,"Index":4,"Attempt":0,"Launch Time":1594091481095,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":37,"Index":5,"Attempt":0,"Launch Time":1594091481100,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":38,"Index":6,"Attempt":0,"Launch Time":1594091481104,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":39,"Index":7,"Attempt":0,"Launch Time":1594091481109,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":40,"Index":8,"Attempt":0,"Launch Time":1594091481112,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":41,"Index":9,"Attempt":0,"Launch Time":1594091481116,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":42,"Index":10,"Attempt":0,"Launch Time":1594091481120,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":43,"Index":11,"Attempt":0,"Launch Time":1594091481123,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":44,"Index":12,"Attempt":0,"Launch Time":1594091481126,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":45,"Index":13,"Attempt":0,"Launch Time":1594091481129,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":46,"Index":14,"Attempt":0,"Launch Time":1594091481132,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":47,"Index":15,"Attempt":0,"Launch Time":1594091481136,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":38,"Index":6,"Attempt":0,"Launch Time":1594091481104,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091482939,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1467","Value":"1467","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"375000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"6250","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":24040,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.resultSerializationTime","Update":11,"Value":11,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":62,"Value":62,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2158,"Value":2158,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":414110000,"Value":414110000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1498,"Value":1498,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":60358000,"Value":60358000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":255,"Value":255,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":255,"Executor Deserialize CPU Time":60358000,"Executor Run Time":1498,"Executor CPU Time":414110000,"Peak Execution Memory":24040,"Result Size":2158,"JVM GC Time":62,"Result Serialization Time":11,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":47,"Index":15,"Attempt":0,"Launch Time":1594091481136,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483014,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1563","Value":"3030","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"750000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"12500","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":48080,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":52,"Value":114,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":4273,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":1324251000,"Value":1738361000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1594,"Value":3092,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":47496000,"Value":107854000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":213,"Value":468,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":213,"Executor Deserialize CPU Time":47496000,"Executor Run Time":1594,"Executor CPU Time":1324251000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":52,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":37,"Index":5,"Attempt":0,"Launch Time":1594091481100,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483015,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1563","Value":"4593","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"1125000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"18750","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":72120,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":62,"Value":176,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":6388,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":507192000,"Value":2245553000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1596,"Value":4688,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":60890000,"Value":168744000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":256,"Value":724,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":256,"Executor Deserialize CPU Time":60890000,"Executor Run Time":1596,"Executor CPU Time":507192000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":62,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":36,"Index":4,"Attempt":0,"Launch Time":1594091481095,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483015,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1564","Value":"6157","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"1500000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"25000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":96160,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":62,"Value":238,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":8503,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":503010000,"Value":2748563000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1596,"Value":6284,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":112849000,"Value":281593000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":260,"Value":984,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":260,"Executor Deserialize CPU Time":112849000,"Executor Run Time":1596,"Executor CPU Time":503010000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":62,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":35,"Index":3,"Attempt":0,"Launch Time":1594091481091,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483016,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1566","Value":"7723","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"1875000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"31250","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":120200,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":62,"Value":300,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":10618,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":502908000,"Value":3251471000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1598,"Value":7882,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":62944000,"Value":344537000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":263,"Value":1247,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":263,"Executor Deserialize CPU Time":62944000,"Executor Run Time":1598,"Executor CPU Time":502908000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":62,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":40,"Index":8,"Attempt":0,"Launch Time":1594091481112,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483016,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1562","Value":"9285","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"2250000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"37500","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":144240,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":62,"Value":362,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":12733,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":510597000,"Value":3762068000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1595,"Value":9477,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":69760000,"Value":414297000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":246,"Value":1493,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":246,"Executor Deserialize CPU Time":69760000,"Executor Run Time":1595,"Executor CPU Time":510597000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":62,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":45,"Index":13,"Attempt":0,"Launch Time":1594091481129,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483024,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1564","Value":"10849","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"2625000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"43750","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":168280,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":62,"Value":424,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":14848,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":495138000,"Value":4257206000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1595,"Value":11072,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":54222000,"Value":468519000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":221,"Value":1714,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":221,"Executor Deserialize CPU Time":54222000,"Executor Run Time":1595,"Executor CPU Time":495138000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":62,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":39,"Index":7,"Attempt":0,"Launch Time":1594091481109,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483024,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1566","Value":"12415","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"3000000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"50000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":192320,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":62,"Value":486,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":16963,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":539451000,"Value":4796657000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1596,"Value":12668,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":64380000,"Value":532899000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":254,"Value":1968,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":254,"Executor Deserialize CPU Time":64380000,"Executor Run Time":1596,"Executor CPU Time":539451000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":62,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":33,"Index":1,"Attempt":0,"Launch Time":1594091481082,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483025,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1566","Value":"13981","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"3375000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"56250","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":216360,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":62,"Value":548,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":19078,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":519178000,"Value":5315835000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1597,"Value":14265,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":54442000,"Value":587341000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":267,"Value":2235,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":267,"Executor Deserialize CPU Time":54442000,"Executor Run Time":1597,"Executor CPU Time":519178000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":62,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":34,"Index":2,"Attempt":0,"Launch Time":1594091481087,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483026,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1574","Value":"15555","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"3750000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"62500","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":240400,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":66,"Value":614,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":21193,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":508433000,"Value":5824268000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1606,"Value":15871,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":69492000,"Value":656833000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":265,"Value":2500,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":265,"Executor Deserialize CPU Time":69492000,"Executor Run Time":1606,"Executor CPU Time":508433000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":66,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":43,"Index":11,"Attempt":0,"Launch Time":1594091481123,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483029,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1578","Value":"17133","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"4125000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"68750","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":264440,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":66,"Value":680,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":23308,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":502120000,"Value":6326388000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1609,"Value":17480,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":48849000,"Value":705682000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":225,"Value":2725,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":225,"Executor Deserialize CPU Time":48849000,"Executor Run Time":1609,"Executor CPU Time":502120000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":66,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":41,"Index":9,"Attempt":0,"Launch Time":1594091481116,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483032,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1583","Value":"18716","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"4500000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"75000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":288480,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":66,"Value":746,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":25423,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":489923000,"Value":6816311000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1614,"Value":19094,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":55787000,"Value":761469000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":240,"Value":2965,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":240,"Executor Deserialize CPU Time":55787000,"Executor Run Time":1614,"Executor CPU Time":489923000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":66,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":46,"Index":14,"Attempt":0,"Launch Time":1594091481132,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483037,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1587","Value":"20303","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"4875000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"81250","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":312520,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":66,"Value":812,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":27538,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":490927000,"Value":7307238000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1618,"Value":20712,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":51464000,"Value":812933000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":218,"Value":3183,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":218,"Executor Deserialize CPU Time":51464000,"Executor Run Time":1618,"Executor CPU Time":490927000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":66,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":32,"Index":0,"Attempt":0,"Launch Time":1594091481077,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483037,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1587","Value":"21890","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"5250000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"87500","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":336560,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":66,"Value":878,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":29653,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":496683000,"Value":7803921000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1619,"Value":22331,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":56827000,"Value":869760000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":271,"Value":3454,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":271,"Executor Deserialize CPU Time":56827000,"Executor Run Time":1619,"Executor CPU Time":496683000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":66,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":42,"Index":10,"Attempt":0,"Launch Time":1594091481120,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483043,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1593","Value":"23483","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"5625000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"93750","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":360600,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":66,"Value":944,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":31768,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":508230000,"Value":8312151000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1624,"Value":23955,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":58152000,"Value":927912000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":229,"Value":3683,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":229,"Executor Deserialize CPU Time":58152000,"Executor Run Time":1624,"Executor CPU Time":508230000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":66,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":44,"Index":12,"Attempt":0,"Launch Time":1594091481126,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483043,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1593","Value":"25076","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"6000000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"100000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":384640,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":66,"Value":1010,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":33883,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":498187000,"Value":8810338000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1624,"Value":25579,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":51988000,"Value":979900000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":222,"Value":3905,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":222,"Executor Deserialize CPU Time":51988000,"Executor Run Time":1624,"Executor CPU Time":498187000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":66,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":2,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":213367864,"JVMOffHeapMemory":189011656,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":2133349,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":2133349,"OffHeapUnifiedMemory":0,"DirectPoolMemory":282024,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":13,"MinorGCTime":115,"MajorGCCount":4,"MajorGCTime":339}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"0","Stage ID":2,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"foreach at :26","Number of Tasks":16,"RDD Info":[{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"mapPartitions\"}","Callsite":"foreach at :26","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"foreach at :26","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"1\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"foreach at :26","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"foreach at :26","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"SQLExecutionRDD","Callsite":"foreach at :26","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.foreach(Dataset.scala:2862)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:26)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:30)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:32)\n$line19.$read$$iw$$iw$$iw$$iw$$iw.(:34)\n$line19.$read$$iw$$iw$$iw$$iw.(:36)\n$line19.$read$$iw$$iw$$iw.(:38)\n$line19.$read$$iw$$iw.(:40)\n$line19.$read$$iw.(:42)\n$line19.$read.(:44)\n$line19.$read$.(:48)\n$line19.$read$.()\n$line19.$eval$.$print$lzycompute(:7)\n$line19.$eval$.$print(:6)\n$line19.$eval.$print()\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)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:745)","Submission Time":1594091481040,"Completion Time":1594091483044,"Accumulables":[{"ID":68,"Name":"internal.metrics.executorCpuTime","Value":8810338000,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.resultSerializationTime","Value":11,"Internal":true,"Count Failed Values":true},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Value":384640,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Value":3905,"Internal":true,"Count Failed Values":true},{"ID":14,"Name":"number of output rows","Value":"100000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":67,"Name":"internal.metrics.executorRunTime","Value":25579,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Value":1010,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"number of output rows","Value":"6000000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":69,"Name":"internal.metrics.resultSize","Value":33883,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Value":979900000,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"duration","Value":"25076","Internal":true,"Count Failed Values":true,"Metadata":"sql"}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":2,"Completion Time":1594091483044,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":0,"time":1594091483045} +{"Event":"SparkListenerApplicationEnd","Timestamp":1594091824231} diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 1f7aa8eec8942..ebdf2f59a2770 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -952,7 +952,7 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu sc.range(0, 1).foreach { _ => new SparkContext(new SparkConf().setAppName("test").setMaster("local") - .set(ALLOW_SPARK_CONTEXT_IN_EXECUTORS, true)).stop() + .set(EXECUTOR_ALLOW_SPARK_CONTEXT, true)).stop() } } } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index d30c1dfea9145..51e38f9cdcd2d 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -178,6 +178,8 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers "executor memory usage" -> "applications/app-20161116163331-0000/executors", "executor resource information" -> "applications/application_1555004656427_0144/executors", "multiple resource profiles" -> "applications/application_1578436911597_0052/environment", + "stage list with peak metrics" -> "applications/app-20200706201101-0003/stages", + "stage with peak metrics" -> "applications/app-20200706201101-0003/stages/2/0", "app environment" -> "applications/app-20161116163331-0000/environment", diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala index 24eb1685f577a..d5829c352be9b 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala @@ -1523,14 +1523,32 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { assert(exec.info.id === id) exec.info.peakMemoryMetrics match { case Some(actual) => - ExecutorMetricType.metricToOffset.foreach { metric => - assert(actual.getMetricValue(metric._1) === metrics.getMetricValue(metric._1)) - } + checkExecutorMetrics(metrics, actual) case _ => assert(false) } } } + + // check stage level executor metrics + val expectedStageValues = Map( + 0 -> StageExecutorMetrics( + new ExecutorMetrics(Array(7000L, 80L, 50L, 20L, 50L, 10L, 100L, 30L, + 80L, 40L, 9000L, 4000L, 8000L, 3000L, 7000L, 2000L)), + Map( + "1" -> new ExecutorMetrics(Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, + 70L, 20L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L)), + "2" -> new ExecutorMetrics(Array(7000L, 80L, 50L, 20L, 10L, 10L, 50L, 30L, + 80L, 40L, 9000L, 4000L, 8000L, 3000L, 7000L, 2000L)))), + 1 -> StageExecutorMetrics( + new ExecutorMetrics(Array(7000L, 70L, 25L, 40L, 60L, 30L, 70L, 60L, + 40L, 20L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L)), + Map( + "1" -> new ExecutorMetrics(Array(7000L, 70L, 20L, 30L, 60L, 30L, 70L, 55L, + 30L, 0L, 5000L, 3000L, 4000L, 2000L, 3000L, 1000L)), + "2" -> new ExecutorMetrics(Array(5500L, 40L, 25L, 40L, 10L, 30L, 35L, 60L, + 40L, 20L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L))))) + checkStageExecutorMetrics(expectedStageValues) } test("stage executor metrics") { @@ -1573,14 +1591,74 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { assert(exec.info.id === id) exec.info.peakMemoryMetrics match { case Some(actual) => - ExecutorMetricType.metricToOffset.foreach { metric => - assert(actual.getMetricValue(metric._1) === metrics.getMetricValue(metric._1)) - } + checkExecutorMetrics(metrics, actual) case _ => assert(false) } } } + + // check stage level executor metrics + val expectedStageValues = Map( + 0 -> StageExecutorMetrics( + new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 50L, 10L, 100L, 30L, + 80L, 40L, 9000L, 4000L, 8000L, 3000L, 7000L, 2000L)), + Map( + "1" -> new ExecutorMetrics(Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, + 70L, 20L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L)), + "2" -> new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, + 80L, 40L, 9000L, 4000L, 8000L, 3000L, 7000L, 2000L)))), + 1 -> StageExecutorMetrics( + new ExecutorMetrics(Array(7000L, 80L, 50L, 40L, 60L, 30L, 80L, 60L, + 50L, 40L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L)), + Map( + "1" -> new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, + 50L, 0L, 5000L, 3000L, 4000L, 2000L, 3000L, 1000L)), + "2" -> new ExecutorMetrics(Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, + 40L, 40L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L))))) + checkStageExecutorMetrics(expectedStageValues) + } + + /** expected stage executor metrics */ + private case class StageExecutorMetrics( + peakExecutorMetrics: ExecutorMetrics, + executorMetrics: Map[String, ExecutorMetrics]) + + private def checkExecutorMetrics(expected: ExecutorMetrics, actual: ExecutorMetrics): Unit = { + ExecutorMetricType.metricToOffset.foreach { metric => + assert(actual.getMetricValue(metric._1) === expected.getMetricValue(metric._1)) + } + } + + /** check stage level peak executor metric values, and executor peak values for each stage */ + private def checkStageExecutorMetrics(expectedStageValues: Map[Int, StageExecutorMetrics]) = { + // check stage level peak executor metric values for each stage + for ((stageId, expectedMetrics) <- expectedStageValues) { + check[StageDataWrapper](Array(stageId, 0)) { stage => + stage.info.peakExecutorMetrics match { + case Some(actual) => + checkExecutorMetrics(expectedMetrics.peakExecutorMetrics, actual) + case None => + assert(false) + } + } + } + + // check peak executor metric values for each stage and executor + val stageExecSummaries = store.view(classOf[ExecutorStageSummaryWrapper]).asScala.toSeq + stageExecSummaries.foreach { exec => + expectedStageValues.get(exec.stageId) match { + case Some(stageValue) => + (stageValue.executorMetrics.get(exec.executorId), exec.info.peakMemoryMetrics) match { + case (Some(expected), Some(actual)) => + checkExecutorMetrics(expected, actual) + case _ => + assert(false) + } + case None => + assert(false) + } + } } test("storage information on executor lost/down") { diff --git a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala index 3d52199b01327..48e0d218c0e5c 100644 --- a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala @@ -93,7 +93,8 @@ class StagePageSuite extends SparkFunSuite with LocalSparkContext { tasks = None, executorSummary = None, killedTasksSummary = Map.empty, - ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID + ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID, + peakExecutorMetrics = None ) val taskTable = new TaskPagedTable( stageData, diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index 2ad4a634cd9a7..56026eaa0072b 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -262,6 +262,27 @@ class UISuite extends SparkFunSuite { } } + test("SPARK-32467: Avoid encoding URL twice on https redirect") { + val (conf, securityMgr, sslOptions) = sslEnabledConf() + val serverInfo = JettyUtils.startJettyServer("0.0.0.0", 0, sslOptions, conf) + try { + val serverAddr = s"http://localhost:${serverInfo.boundPort}" + + val (_, ctx) = newContext("/ctx1") + serverInfo.addHandler(ctx, securityMgr) + + TestUtils.withHttpConnection(new URL(s"$serverAddr/ctx%281%29?a%5B0%5D=b")) { conn => + assert(conn.getResponseCode() === HttpServletResponse.SC_FOUND) + val location = Option(conn.getHeaderFields().get("Location")) + .map(_.get(0)).orNull + val expectedLocation = s"https://localhost:${serverInfo.securePort.get}/ctx(1)?a[0]=b" + assert(location == expectedLocation) + } + } finally { + stopServer(serverInfo) + } + } + test("http -> https redirect applies to all URIs") { val (conf, securityMgr, sslOptions) = sslEnabledConf() val serverInfo = JettyUtils.startJettyServer("0.0.0.0", 0, sslOptions, conf) diff --git a/dev/.rat-excludes b/dev/.rat-excludes index 3889dc9ba3d3a..df1dd51a7c519 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -123,3 +123,4 @@ SessionHandler.java GangliaReporter.java application_1578436911597_0052 config.properties +app-20200706201101-0003 diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 index 45c346fa5ecc4..4936d808db85a 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 @@ -156,7 +156,7 @@ metrics-jmx/4.1.1//metrics-jmx-4.1.1.jar metrics-json/4.1.1//metrics-json-4.1.1.jar metrics-jvm/4.1.1//metrics-jvm-4.1.1.jar minlog/1.3.0//minlog-1.3.0.jar -netty-all/4.1.47.Final//netty-all-4.1.47.Final.jar +netty-all/4.1.51.Final//netty-all-4.1.51.Final.jar objenesis/2.6//objenesis-2.6.jar okhttp/3.12.6//okhttp-3.12.6.jar okio/1.14.0//okio-1.14.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 index 4dc36d65f8632..82cd20bf77191 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 @@ -171,7 +171,7 @@ metrics-jmx/4.1.1//metrics-jmx-4.1.1.jar metrics-json/4.1.1//metrics-json-4.1.1.jar metrics-jvm/4.1.1//metrics-jvm-4.1.1.jar minlog/1.3.0//minlog-1.3.0.jar -netty-all/4.1.47.Final//netty-all-4.1.47.Final.jar +netty-all/4.1.51.Final//netty-all-4.1.51.Final.jar objenesis/2.6//objenesis-2.6.jar okhttp/3.12.6//okhttp-3.12.6.jar okio/1.14.0//okio-1.14.0.jar diff --git a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 index 40feed25d3a6d..7257fb8722422 100644 --- a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 @@ -183,7 +183,7 @@ metrics-jmx/4.1.1//metrics-jmx-4.1.1.jar metrics-json/4.1.1//metrics-json-4.1.1.jar metrics-jvm/4.1.1//metrics-jvm-4.1.1.jar minlog/1.3.0//minlog-1.3.0.jar -netty-all/4.1.47.Final//netty-all-4.1.47.Final.jar +netty-all/4.1.51.Final//netty-all-4.1.51.Final.jar nimbus-jose-jwt/4.41.1//nimbus-jose-jwt-4.41.1.jar objenesis/2.6//objenesis-2.6.jar okhttp/2.7.5//okhttp-2.7.5.jar diff --git a/docs/core-migration-guide.md b/docs/core-migration-guide.md index b2a08502d0d6f..11d3e0019617f 100644 --- a/docs/core-migration-guide.md +++ b/docs/core-migration-guide.md @@ -24,7 +24,7 @@ license: | ## Upgrading from Core 3.0 to 3.1 -- In Spark 3.0 and below, `SparkContext` can be created in executors. Since Spark 3.1, an exception will be thrown when creating `SparkContext` in executors. You can allow it by setting the configuration `spark.driver.allowSparkContextInExecutors` when creating `SparkContext` in executors. +- In Spark 3.0 and below, `SparkContext` can be created in executors. Since Spark 3.1, an exception will be thrown when creating `SparkContext` in executors. You can allow it by setting the configuration `spark.executor.allowSparkContext` when creating `SparkContext` in executors. ## Upgrading from Core 2.4 to 3.0 diff --git a/docs/monitoring.md b/docs/monitoring.md index 2ab7b30a1dca9..247957d087fa3 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -573,6 +573,7 @@ can be identified by their `[attempt-id]`. In the API listed below, when running ?planDescription=[true (default) | false] enables/disables Physical planDescription on demand when Physical Plan size is high.
?offset=[offset]&length=[len] lists queries in the given range. + /applications/[app-id]/sql/[execution-id] @@ -581,6 +582,7 @@ can be identified by their `[attempt-id]`. In the API listed below, when running ?details=[true (default) | false] lists/hides metric details in addition to given query details.
?planDescription=[true (default) | false] enables/disables Physical planDescription on demand for the given query when Physical Plan size is high. + /applications/[app-id]/environment diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 7c9c0a726c881..f646c09438ad1 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -24,14 +24,16 @@ license: | ## Upgrading from Spark SQL 3.0 to 3.1 - - In Spark 3.1, grouping_id() returns long values. In Spark version 3.0 and earlier, this function returns int values. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.integerGroupingId` to `true`. + - In Spark 3.1, grouping_id() returns long values. In Spark version 3.0 and earlier, this function returns int values. To restore the behavior before Spark 3.1, you can set `spark.sql.legacy.integerGroupingId` to `true`. - - In Spark 3.1, SQL UI data adopts the `formatted` mode for the query plan explain results. To restore the behavior before Spark 3.0, you can set `spark.sql.ui.explainMode` to `extended`. + - In Spark 3.1, SQL UI data adopts the `formatted` mode for the query plan explain results. To restore the behavior before Spark 3.1, you can set `spark.sql.ui.explainMode` to `extended`. - In Spark 3.1, `from_unixtime`, `unix_timestamp`,`to_unix_timestamp`, `to_timestamp` and `to_date` will fail if the specified datetime pattern is invalid. In Spark 3.0 or earlier, they result `NULL`. - In Spark 3.1, the Parquet, ORC, Avro and JSON datasources throw the exception `org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the data schema` in read if they detect duplicate names in top-level columns as well in nested structures. The datasources take into account the SQL config `spark.sql.caseSensitive` while detecting column name duplicates. + - In Spark 3.1, structs and maps are wrapped by the `{}` brackets in casting them to strings. For instance, the `show()` action and the `CAST` expression use such brackets. In Spark 3.0 and earlier, the `[]` brackets are used for the same purpose. To restore the behavior before Spark 3.1, you can set `spark.sql.legacy.castComplexTypesToString.enabled` to `true`. + ## Upgrading from Spark SQL 3.0 to 3.0.1 - In Spark 3.0, JSON datasource and JSON function `schema_of_json` infer TimestampType from string values if they match to the pattern defined by the JSON option `timestampFormat`. Since version 3.0.1, the timestamp type inference is disabled by default. Set the JSON option `inferTimestamp` to `true` to enable such type inference. diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala index 099d6ff13051b..14dcbeef0d9a3 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala @@ -194,6 +194,7 @@ class KafkaContinuousSourceTopicDeletionSuite extends KafkaContinuousTest { .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") .option("kafka.request.timeout.ms", "3000") + .option("kafka.default.api.timeout.ms", "3000") .option("subscribePattern", s"$topicPrefix-.*") .option("failOnDataLoss", "false") diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala index 5b634e4d50641..54ce1717acc71 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala @@ -224,6 +224,7 @@ class KafkaSourceStressForDontFailOnDataLossSuite extends StreamTest with KafkaM .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") .option("kafka.request.timeout.ms", "3000") + .option("kafka.default.api.timeout.ms", "3000") .option("subscribePattern", "failOnDataLoss.*") .option("startingOffsets", "earliest") .option("failOnDataLoss", "false") diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala index 517d153ca3c91..63659989dec1b 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala @@ -364,6 +364,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") .option("kafka.request.timeout.ms", "3000") + .option("kafka.default.api.timeout.ms", "3000") .option("subscribePattern", s"$topicPrefix-.*") .option("failOnDataLoss", "false") @@ -401,6 +402,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") .option("kafka.request.timeout.ms", "3000") + .option("kafka.default.api.timeout.ms", "3000") .option("startingOffsets", "earliest") .option("subscribePattern", s"$topicPrefix-.*") @@ -590,6 +592,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") .option("kafka.request.timeout.ms", "3000") + .option("kafka.default.api.timeout.ms", "3000") .option("subscribe", topic) // If a topic is deleted and we try to poll data starting from offset 0, // the Kafka consumer will just block until timeout and return an empty result. @@ -1861,6 +1864,7 @@ class KafkaSourceStressSuite extends KafkaSourceTest { .option("subscribePattern", "stress.*") .option("failOnDataLoss", "false") .option("kafka.request.timeout.ms", "3000") + .option("kafka.default.api.timeout.ms", "3000") .load() .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") .as[(String, String)] diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala index 64f1722f5fcb8..ad1010da5c104 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala @@ -64,6 +64,8 @@ private[feature] trait ImputerParams extends Params with HasInputCol with HasInp /** @group getParam */ def getMissingValue: Double = $(missingValue) + setDefault(strategy -> Imputer.mean, missingValue -> Double.NaN) + /** Returns the input and output column names corresponding in pair. */ private[feature] def getInOutCols(): (Array[String], Array[String]) = { if (isSet(inputCol)) { @@ -144,8 +146,6 @@ class Imputer @Since("2.2.0") (@Since("2.2.0") override val uid: String) @Since("3.0.0") def setRelativeError(value: Double): this.type = set(relativeError, value) - setDefault(strategy -> Imputer.mean, missingValue -> Double.NaN) - override def fit(dataset: Dataset[_]): ImputerModel = { transformSchema(dataset.schema, logging = true) val spark = dataset.sparkSession diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala index 90187c331e835..22c4ca9cddf4b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala @@ -58,6 +58,8 @@ private[feature] trait MinMaxScalerParams extends Params with HasInputCol with H /** @group getParam */ def getMax: Double = $(max) + setDefault(min -> 0.0, max -> 1.0) + /** Validates and transforms the input schema. */ protected def validateAndTransformSchema(schema: StructType): StructType = { require($(min) < $(max), s"The specified min(${$(min)}) is larger or equal to max(${$(max)})") @@ -93,8 +95,6 @@ class MinMaxScaler @Since("1.5.0") (@Since("1.5.0") override val uid: String) @Since("1.5.0") def this() = this(Identifiable.randomUID("minMaxScal")) - setDefault(min -> 0.0, max -> 1.0) - /** @group setParam */ @Since("1.5.0") def setInputCol(value: String): this.type = set(inputCol, value) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala index 0ef092f6be463..5a500fefb57ec 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala @@ -53,8 +53,6 @@ private[ml] trait OneHotEncoderBase extends Params with HasHandleInvalid "during fitting, invalid data will result in an error.", ParamValidators.inArray(OneHotEncoder.supportedHandleInvalids)) - setDefault(handleInvalid, OneHotEncoder.ERROR_INVALID) - /** * Whether to drop the last category in the encoded vector (default: true) * @group param @@ -62,12 +60,13 @@ private[ml] trait OneHotEncoderBase extends Params with HasHandleInvalid @Since("2.3.0") final val dropLast: BooleanParam = new BooleanParam(this, "dropLast", "whether to drop the last category") - setDefault(dropLast -> true) /** @group getParam */ @Since("2.3.0") def getDropLast: Boolean = $(dropLast) + setDefault(handleInvalid -> OneHotEncoder.ERROR_INVALID, dropLast -> true) + /** Returns the input and output column names corresponding in pair. */ private[feature] def getInOutCols(): (Array[String], Array[String]) = { if (isSet(inputCol)) { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala index 4eedfc4dc0efa..b93c9b1fcd204 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala @@ -46,7 +46,6 @@ private[feature] trait QuantileDiscretizerBase extends Params val numBuckets = new IntParam(this, "numBuckets", "Number of buckets (quantiles, or " + "categories) into which data points are grouped. Must be >= 2.", ParamValidators.gtEq(2)) - setDefault(numBuckets -> 2) /** @group getParam */ def getNumBuckets: Int = getOrDefault(numBuckets) @@ -82,7 +81,8 @@ private[feature] trait QuantileDiscretizerBase extends Params "how to handle invalid entries. Options are skip (filter out rows with invalid values), " + "error (throw an error), or keep (keep invalid values in a special additional bucket).", ParamValidators.inArray(Bucketizer.supportedHandleInvalids)) - setDefault(handleInvalid, Bucketizer.ERROR_INVALID) + + setDefault(handleInvalid -> Bucketizer.ERROR_INVALID, numBuckets -> 2) } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala index 7ccfafa4ac813..b8da020017f12 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala @@ -60,7 +60,6 @@ private[feature] trait RFormulaBase extends HasFeaturesCol with HasLabelCol with @Since("2.1.0") val forceIndexLabel: BooleanParam = new BooleanParam(this, "forceIndexLabel", "Force to index label whether it is numeric or string") - setDefault(forceIndexLabel -> false) /** @group getParam */ @Since("2.1.0") @@ -80,7 +79,6 @@ private[feature] trait RFormulaBase extends HasFeaturesCol with HasLabelCol with "type. Options are 'skip' (filter out rows with invalid data), error (throw an error), " + "or 'keep' (put invalid data in a special additional bucket, at index numLabels).", ParamValidators.inArray(StringIndexer.supportedHandleInvalids)) - setDefault(handleInvalid, StringIndexer.ERROR_INVALID) /** * Param for how to order categories of a string FEATURE column used by `StringIndexer`. @@ -113,12 +111,14 @@ private[feature] trait RFormulaBase extends HasFeaturesCol with HasLabelCol with "The default value is 'frequencyDesc'. When the ordering is set to 'alphabetDesc', " + "RFormula drops the same category as R when encoding strings.", ParamValidators.inArray(StringIndexer.supportedStringOrderType)) - setDefault(stringIndexerOrderType, StringIndexer.frequencyDesc) /** @group getParam */ @Since("2.3.0") def getStringIndexerOrderType: String = $(stringIndexerOrderType) + setDefault(forceIndexLabel -> false, handleInvalid -> StringIndexer.ERROR_INVALID, + stringIndexerOrderType -> StringIndexer.frequencyDesc) + protected def hasLabelCol(schema: StructType): Boolean = { schema.map(_.name).contains($(labelCol)) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala index 72ab3dbc31016..e8f325ec58432 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala @@ -50,8 +50,6 @@ private[feature] trait RobustScalerParams extends Params with HasInputCol with H /** @group getParam */ def getLower: Double = $(lower) - setDefault(lower -> 0.25) - /** * Upper quantile to calculate quantile range, shared by all features * Default: 0.75 @@ -64,8 +62,6 @@ private[feature] trait RobustScalerParams extends Params with HasInputCol with H /** @group getParam */ def getUpper: Double = $(upper) - setDefault(upper -> 0.75) - /** * Whether to center the data with median before scaling. * It will build a dense output, so take care when applying to sparse input. @@ -78,8 +74,6 @@ private[feature] trait RobustScalerParams extends Params with HasInputCol with H /** @group getParam */ def getWithCentering: Boolean = $(withCentering) - setDefault(withCentering -> false) - /** * Whether to scale the data to quantile range. * Default: true @@ -91,7 +85,7 @@ private[feature] trait RobustScalerParams extends Params with HasInputCol with H /** @group getParam */ def getWithScaling: Boolean = $(withScaling) - setDefault(withScaling -> true) + setDefault(withScaling -> true, lower -> 0.25, upper -> 0.75, withCentering -> false) /** Validates and transforms the input schema. */ protected def validateAndTransformSchema(schema: StructType): StructType = { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Selector.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Selector.scala index 627133968d149..46052a89fdf1a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Selector.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Selector.scala @@ -50,7 +50,6 @@ private[feature] trait SelectorParams extends Params "Number of features that selector will select, ordered by ascending p-value. If the" + " number of features is < numTopFeatures, then this will select all features.", ParamValidators.gtEq(1)) - setDefault(numTopFeatures -> 50) /** @group getParam */ @Since("3.1.0") @@ -66,7 +65,6 @@ private[feature] trait SelectorParams extends Params final val percentile = new DoubleParam(this, "percentile", "Percentile of features that selector will select, ordered by ascending p-value.", ParamValidators.inRange(0, 1)) - setDefault(percentile -> 0.1) /** @group getParam */ @Since("3.1.0") @@ -81,7 +79,6 @@ private[feature] trait SelectorParams extends Params @Since("3.1.0") final val fpr = new DoubleParam(this, "fpr", "The higest p-value for features to be kept.", ParamValidators.inRange(0, 1)) - setDefault(fpr -> 0.05) /** @group getParam */ @Since("3.1.0") @@ -96,7 +93,6 @@ private[feature] trait SelectorParams extends Params @Since("3.1.0") final val fdr = new DoubleParam(this, "fdr", "The upper bound of the expected false discovery rate.", ParamValidators.inRange(0, 1)) - setDefault(fdr -> 0.05) /** @group getParam */ def getFdr: Double = $(fdr) @@ -110,7 +106,6 @@ private[feature] trait SelectorParams extends Params @Since("3.1.0") final val fwe = new DoubleParam(this, "fwe", "The upper bound of the expected family-wise error rate.", ParamValidators.inRange(0, 1)) - setDefault(fwe -> 0.05) /** @group getParam */ def getFwe: Double = $(fwe) @@ -125,12 +120,13 @@ private[feature] trait SelectorParams extends Params "The selector type. Supported options: numTopFeatures, percentile, fpr, fdr, fwe", ParamValidators.inArray(Array("numTopFeatures", "percentile", "fpr", "fdr", "fwe"))) - setDefault(selectorType -> "numTopFeatures") /** @group getParam */ @Since("3.1.0") def getSelectorType: String = $(selectorType) + setDefault(numTopFeatures -> 50, percentile -> 0.1, fpr -> 0.05, fdr -> 0.05, fwe -> 0.05, + selectorType -> "numTopFeatures") } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala index be32f44287b6a..ab51fe6e78bd7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala @@ -56,8 +56,6 @@ private[feature] trait StringIndexerBase extends Params with HasHandleInvalid wi "or 'keep' (put invalid data in a special additional bucket, at index numLabels).", ParamValidators.inArray(StringIndexer.supportedHandleInvalids)) - setDefault(handleInvalid, StringIndexer.ERROR_INVALID) - /** * Param for how to order labels of string column. The first label after ordering is assigned * an index of 0. @@ -80,6 +78,9 @@ private[feature] trait StringIndexerBase extends Params with HasHandleInvalid wi s"Supported options: ${StringIndexer.supportedStringOrderType.mkString(", ")}.", ParamValidators.inArray(StringIndexer.supportedStringOrderType)) + setDefault(handleInvalid -> StringIndexer.ERROR_INVALID, + stringOrderType -> StringIndexer.frequencyDesc) + /** @group getParam */ @Since("2.3.0") def getStringOrderType: String = $(stringOrderType) @@ -155,7 +156,6 @@ class StringIndexer @Since("1.4.0") ( /** @group setParam */ @Since("2.3.0") def setStringOrderType(value: String): this.type = set(stringOrderType, value) - setDefault(stringOrderType, StringIndexer.frequencyDesc) /** @group setParam */ @Since("1.4.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala index 866074fb1453e..b7cf4392cd177 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala @@ -60,8 +60,6 @@ private[ml] trait VectorIndexerParams extends Params with HasInputCol with HasOu "number of categories of the feature).", ParamValidators.inArray(VectorIndexer.supportedHandleInvalids)) - setDefault(handleInvalid, VectorIndexer.ERROR_INVALID) - /** * Threshold for the number of values a categorical feature can take. * If a feature is found to have {@literal >} maxCategories values, then it is declared @@ -75,10 +73,10 @@ private[ml] trait VectorIndexerParams extends Params with HasInputCol with HasOu " If a feature is found to have > maxCategories values, then it is declared continuous.", ParamValidators.gtEq(2)) - setDefault(maxCategories -> 20) - /** @group getParam */ def getMaxCategories: Int = $(maxCategories) + + setDefault(maxCategories -> 20, handleInvalid -> VectorIndexer.ERROR_INVALID) } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala index be91844ba39e6..3590b9118f3b8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala @@ -57,8 +57,6 @@ final class VectorSlicer @Since("1.5.0") (@Since("1.5.0") override val uid: Stri "An array of indices to select features from a vector column." + " There can be no overlap with names.", VectorSlicer.validIndices) - setDefault(indices -> Array.emptyIntArray) - /** @group getParam */ @Since("1.5.0") def getIndices: Array[Int] = $(indices) @@ -79,8 +77,6 @@ final class VectorSlicer @Since("1.5.0") (@Since("1.5.0") override val uid: Stri "An array of feature names to select features from a vector column." + " There can be no overlap with indices.", VectorSlicer.validNames) - setDefault(names -> Array.empty[String]) - /** @group getParam */ @Since("1.5.0") def getNames: Array[String] = $(names) @@ -97,6 +93,8 @@ final class VectorSlicer @Since("1.5.0") (@Since("1.5.0") override val uid: Stri @Since("1.5.0") def setOutputCol(value: String): this.type = set(outputCol, value) + setDefault(indices -> Array.emptyIntArray, names -> Array.empty[String]) + @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { // Validity checks diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala index db2665fa2e4a3..01db39f9e3921 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala @@ -47,7 +47,6 @@ private[feature] trait Word2VecBase extends Params final val vectorSize = new IntParam( this, "vectorSize", "the dimension of codes after transforming from words (> 0)", ParamValidators.gt(0)) - setDefault(vectorSize -> 100) /** @group getParam */ def getVectorSize: Int = $(vectorSize) @@ -60,7 +59,6 @@ private[feature] trait Word2VecBase extends Params final val windowSize = new IntParam( this, "windowSize", "the window size (context words from [-window, window]) (> 0)", ParamValidators.gt(0)) - setDefault(windowSize -> 5) /** @group expertGetParam */ def getWindowSize: Int = $(windowSize) @@ -73,7 +71,6 @@ private[feature] trait Word2VecBase extends Params final val numPartitions = new IntParam( this, "numPartitions", "number of partitions for sentences of words (> 0)", ParamValidators.gt(0)) - setDefault(numPartitions -> 1) /** @group getParam */ def getNumPartitions: Int = $(numPartitions) @@ -86,7 +83,6 @@ private[feature] trait Word2VecBase extends Params */ final val minCount = new IntParam(this, "minCount", "the minimum number of times a token must " + "appear to be included in the word2vec model's vocabulary (>= 0)", ParamValidators.gtEq(0)) - setDefault(minCount -> 5) /** @group getParam */ def getMinCount: Int = $(minCount) @@ -101,13 +97,12 @@ private[feature] trait Word2VecBase extends Params final val maxSentenceLength = new IntParam(this, "maxSentenceLength", "Maximum length " + "(in words) of each sentence in the input data. Any sentence longer than this threshold will " + "be divided into chunks up to the size (> 0)", ParamValidators.gt(0)) - setDefault(maxSentenceLength -> 1000) /** @group getParam */ def getMaxSentenceLength: Int = $(maxSentenceLength) - setDefault(stepSize -> 0.025) - setDefault(maxIter -> 1) + setDefault(vectorSize -> 100, windowSize -> 5, numPartitions -> 1, minCount -> 5, + maxSentenceLength -> 1000, stepSize -> 0.025, maxIter -> 1) /** * Validate and transform the input schema. diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala index 7e2c287f146fb..19ea8ae4775d8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala @@ -333,8 +333,6 @@ private[ml] trait TreeEnsembleParams extends DecisionTreeParams { "Fraction of the training data used for learning each decision tree, in range (0, 1].", ParamValidators.inRange(0, 1, lowerInclusive = false, upperInclusive = true)) - setDefault(subsamplingRate -> 1.0) - /** @group getParam */ final def getSubsamplingRate: Double = $(subsamplingRate) @@ -386,10 +384,10 @@ private[ml] trait TreeEnsembleParams extends DecisionTreeParams { || Try(value.toInt).filter(_ > 0).isSuccess || Try(value.toDouble).filter(_ > 0).filter(_ <= 1.0).isSuccess) - setDefault(featureSubsetStrategy -> "auto") - /** @group getParam */ final def getFeatureSubsetStrategy: String = $(featureSubsetStrategy).toLowerCase(Locale.ROOT) + + setDefault(subsamplingRate -> 1.0, featureSubsetStrategy -> "auto") } /** @@ -448,8 +446,6 @@ private[ml] trait RandomForestParams extends TreeEnsembleParams { new IntParam(this, "numTrees", "Number of trees to train (at least 1)", ParamValidators.gtEq(1)) - setDefault(numTrees -> 20) - /** @group getParam */ final def getNumTrees: Int = $(numTrees) @@ -461,11 +457,11 @@ private[ml] trait RandomForestParams extends TreeEnsembleParams { final val bootstrap: BooleanParam = new BooleanParam(this, "bootstrap", "Whether bootstrap samples are used when building trees.") - setDefault(bootstrap -> true) - /** @group getParam */ @Since("3.0.0") final def getBootstrap: Boolean = $(bootstrap) + + setDefault(numTrees -> 20, bootstrap -> true) } private[ml] trait RandomForestClassifierParams @@ -518,9 +514,7 @@ private[ml] trait GBTParams extends TreeEnsembleParams with HasMaxIter with HasS "(a.k.a. learning rate) in interval (0, 1] for shrinking the contribution of each estimator.", ParamValidators.inRange(0, 1, lowerInclusive = false, upperInclusive = true)) - setDefault(maxIter -> 20, stepSize -> 0.1, validationTol -> 0.01) - - setDefault(featureSubsetStrategy -> "all") + setDefault(maxIter -> 20, stepSize -> 0.1, validationTol -> 0.01, featureSubsetStrategy -> "all") /** (private[ml]) Create a BoostingStrategy instance to use with the old API. */ private[ml] def getOldBoostingStrategy( diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala index e99c55b0cdd85..275d3c5510f7d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala @@ -55,8 +55,6 @@ private[ml] trait CrossValidatorParams extends ValidatorParams { /** @group getParam */ def getNumFolds: Int = $(numFolds) - setDefault(numFolds -> 3) - /** * Param for the column name of user specified fold number. Once this is specified, * `CrossValidator` won't do random k-fold split. Note that this column should be @@ -68,7 +66,7 @@ private[ml] trait CrossValidatorParams extends ValidatorParams { def getFoldCol: String = $(foldCol) - setDefault(foldCol, "") + setDefault(foldCol -> "", numFolds -> 3) } /** diff --git a/mllib/src/test/scala/org/apache/spark/ml/util/DefaultReadWriteTest.scala b/mllib/src/test/scala/org/apache/spark/ml/util/DefaultReadWriteTest.scala index 4d9e664850c12..dd0139b94f098 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/util/DefaultReadWriteTest.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/util/DefaultReadWriteTest.scala @@ -63,6 +63,9 @@ trait DefaultReadWriteTest extends TempDirectory { self: Suite => (instance.getOrDefault(p), newInstance.getOrDefault(p)) match { case (Array(values), Array(newValues)) => assert(values === newValues, s"Values do not match on param ${p.name}.") + case (value: Double, newValue: Double) => + assert(value.isNaN && newValue.isNaN || value == newValue, + s"Values do not match on param ${p.name}.") case (value, newValue) => assert(value === newValue, s"Values do not match on param ${p.name}.") } diff --git a/pom.xml b/pom.xml index d07813c2787de..6b72e7d13643b 100644 --- a/pom.xml +++ b/pom.xml @@ -726,7 +726,7 @@ io.netty netty-all - 4.1.47.Final + 4.1.51.Final org.apache.derby diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 0816657692826..55a5657b64055 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -118,7 +118,7 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, ValueError:... """ if (conf is None or - conf.get("spark.driver.allowSparkContextInExecutors", "false").lower() != "true"): + conf.get("spark.executor.allowSparkContext", "false").lower() != "true"): # In order to prevent SparkContext from being created in executors. SparkContext._assert_on_driver() diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 3bc9dc0628aa8..a223a9be4d8ea 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -515,8 +515,8 @@ class _LinearSVCParams(_ClassifierParams, HasRegParam, HasMaxIter, HasFitInterce " all predictions 0.0 and -Inf will make all predictions 1.0.", typeConverter=TypeConverters.toFloat) - def __init__(self): - super(_LinearSVCParams, self).__init__() + def __init__(self, *args): + super(_LinearSVCParams, self).__init__(*args) self._setDefault(maxIter=100, regParam=0.0, tol=1e-6, fitIntercept=True, standardization=True, threshold=0.0, aggregationDepth=2, blockSize=1) @@ -589,6 +589,8 @@ class LinearSVC(_JavaClassifier, _LinearSVCParams, JavaMLWritable, JavaMLReadabl True >>> model.intercept == model2.intercept True + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True .. versionadded:: 2.2.0 """ @@ -822,8 +824,8 @@ class _LogisticRegressionParams(_ProbabilisticClassifierParams, HasRegParam, "classes for multinomial regression.", typeConverter=TypeConverters.toVector) - def __init__(self): - super(_LogisticRegressionParams, self).__init__() + def __init__(self, *args): + super(_LogisticRegressionParams, self).__init__(*args) self._setDefault(maxIter=100, regParam=0.0, tol=1E-6, threshold=0.5, family="auto", blockSize=1) @@ -1020,6 +1022,8 @@ class LogisticRegression(_JavaProbabilisticClassifier, _LogisticRegressionParams True >>> model2 LogisticRegressionModel: uid=..., numClasses=2, numFeatures=2 + >>> blorModel.transform(test0).take(1) == model2.transform(test0).take(1) + True .. versionadded:: 1.3.0 """ @@ -1315,8 +1319,8 @@ class _DecisionTreeClassifierParams(_DecisionTreeParams, _TreeClassifierParams): Params for :py:class:`DecisionTreeClassifier` and :py:class:`DecisionTreeClassificationModel`. """ - def __init__(self): - super(_DecisionTreeClassifierParams, self).__init__() + def __init__(self, *args): + super(_DecisionTreeClassifierParams, self).__init__(*args) self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="gini", leafCol="", minWeightFractionPerNode=0.0) @@ -1386,7 +1390,8 @@ class DecisionTreeClassifier(_JavaProbabilisticClassifier, _DecisionTreeClassifi >>> model2 = DecisionTreeClassificationModel.load(model_path) >>> model.featureImportances == model2.featureImportances True - + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True >>> df3 = spark.createDataFrame([ ... (1.0, 0.2, Vectors.dense(1.0)), ... (1.0, 0.8, Vectors.dense(1.0)), @@ -1552,8 +1557,8 @@ class _RandomForestClassifierParams(_RandomForestParams, _TreeClassifierParams): Params for :py:class:`RandomForestClassifier` and :py:class:`RandomForestClassificationModel`. """ - def __init__(self): - super(_RandomForestClassifierParams, self).__init__() + def __init__(self, *args): + super(_RandomForestClassifierParams, self).__init__(*args) self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="gini", numTrees=20, featureSubsetStrategy="auto", @@ -1630,6 +1635,8 @@ class RandomForestClassifier(_JavaProbabilisticClassifier, _RandomForestClassifi >>> model2 = RandomForestClassificationModel.load(model_path) >>> model.featureImportances == model2.featureImportances True + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True .. versionadded:: 1.4.0 """ @@ -1895,8 +1902,8 @@ class _GBTClassifierParams(_GBTParams, _HasVarianceImpurity): "Supported options: " + ", ".join(supportedLossTypes), typeConverter=TypeConverters.toString) - def __init__(self): - super(_GBTClassifierParams, self).__init__() + def __init__(self, *args): + super(_GBTClassifierParams, self).__init__(*args) self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, lossType="logistic", maxIter=20, stepSize=0.1, subsamplingRate=1.0, @@ -1994,6 +2001,8 @@ class GBTClassifier(_JavaProbabilisticClassifier, _GBTClassifierParams, True >>> model.treeWeights == model2.treeWeights True + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True >>> model.trees [DecisionTreeRegressionModel...depth=..., DecisionTreeRegressionModel...] >>> validation = spark.createDataFrame([(0.0, Vectors.dense(-1.0),)], @@ -2227,8 +2236,8 @@ class _NaiveBayesParams(_PredictorParams, HasWeightCol): "and gaussian.", typeConverter=TypeConverters.toString) - def __init__(self): - super(_NaiveBayesParams, self).__init__() + def __init__(self, *args): + super(_NaiveBayesParams, self).__init__(*args) self._setDefault(smoothing=1.0, modelType="multinomial") @since("1.5.0") @@ -2314,6 +2323,8 @@ class NaiveBayes(_JavaProbabilisticClassifier, _NaiveBayesParams, HasThresholds, True >>> model.theta == model2.theta True + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True >>> nb = nb.setThresholds([0.01, 10.00]) >>> model3 = nb.fit(df) >>> result = model3.transform(test0).head() @@ -2440,8 +2451,8 @@ class _MultilayerPerceptronParams(_ProbabilisticClassifierParams, HasSeed, HasMa initialWeights = Param(Params._dummy(), "initialWeights", "The initial weights of the model.", typeConverter=TypeConverters.toVector) - def __init__(self): - super(_MultilayerPerceptronParams, self).__init__() + def __init__(self, *args): + super(_MultilayerPerceptronParams, self).__init__(*args) self._setDefault(maxIter=100, tol=1E-6, blockSize=128, stepSize=0.03, solver="l-bfgs") @since("1.6.0") @@ -2523,6 +2534,8 @@ class MultilayerPerceptronClassifier(_JavaProbabilisticClassifier, _MultilayerPe True >>> model.weights == model2.weights True + >>> model.transform(testDF).take(1) == model2.transform(testDF).take(1) + True >>> mlp2 = mlp2.setInitialWeights(list(range(0, 12))) >>> model3 = mlp2.fit(df) >>> model3.weights != model2.weights @@ -2742,6 +2755,8 @@ class OneVsRest(Estimator, _OneVsRestParams, HasParallelism, JavaMLReadable, Jav >>> model2 = OneVsRestModel.load(model_path) >>> model2.transform(test0).head().newPrediction 0.0 + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True >>> model.transform(test2).columns ['features', 'rawPrediction', 'newPrediction'] @@ -3167,6 +3182,17 @@ class FMClassifier(_JavaProbabilisticClassifier, _FactorizationMachinesParams, J DenseVector([14.8232]) >>> model.factors DenseMatrix(1, 2, [0.0163, -0.0051], 1) + >>> model_path = temp_path + "/fm_model" + >>> model.save(model_path) + >>> model2 = FMClassificationModel.load(model_path) + >>> model2.intercept + -7.316665276826291 + >>> model2.linear + DenseVector([14.8232]) + >>> model2.factors + DenseMatrix(1, 2, [0.0163, -0.0051], 1) + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True .. versionadded:: 3.0.0 """ diff --git a/python/pyspark/ml/clustering.py b/python/pyspark/ml/clustering.py index 6ca413d696368..2d70f876849f8 100644 --- a/python/pyspark/ml/clustering.py +++ b/python/pyspark/ml/clustering.py @@ -109,8 +109,8 @@ class _GaussianMixtureParams(HasMaxIter, HasFeaturesCol, HasSeed, HasPredictionC k = Param(Params._dummy(), "k", "Number of independent Gaussians in the mixture model. " + "Must be > 1.", typeConverter=TypeConverters.toInt) - def __init__(self): - super(_GaussianMixtureParams, self).__init__() + def __init__(self, *args): + super(_GaussianMixtureParams, self).__init__(*args) self._setDefault(k=2, tol=0.01, maxIter=100, aggregationDepth=2, blockSize=1) @since("2.0.0") @@ -325,6 +325,8 @@ class GaussianMixture(JavaEstimator, _GaussianMixtureParams, JavaMLWritable, Jav Row(mean=DenseVector([0.825, 0.8675])) >>> model2.gaussiansDF.select("cov").head() Row(cov=DenseMatrix(2, 2, [0.0056, -0.0051, -0.0051, 0.0046], False)) + >>> model.transform(df).take(1) == model2.transform(df).take(1) + True >>> gm2.setWeightCol("weight") GaussianMixture... @@ -503,8 +505,8 @@ class _KMeansParams(HasMaxIter, HasFeaturesCol, HasSeed, HasPredictionCol, HasTo initSteps = Param(Params._dummy(), "initSteps", "The number of steps for k-means|| " + "initialization mode. Must be > 0.", typeConverter=TypeConverters.toInt) - def __init__(self): - super(_KMeansParams, self).__init__() + def __init__(self, *args): + super(_KMeansParams, self).__init__(*args) self._setDefault(k=2, initMode="k-means||", initSteps=2, tol=1e-4, maxIter=20, distanceMeasure="euclidean") @@ -637,6 +639,8 @@ class KMeans(JavaEstimator, _KMeansParams, JavaMLWritable, JavaMLReadable): array([ True, True], dtype=bool) >>> model.clusterCenters()[1] == model2.clusterCenters()[1] array([ True, True], dtype=bool) + >>> model.transform(df).take(1) == model2.transform(df).take(1) + True .. versionadded:: 1.5.0 """ @@ -760,8 +764,8 @@ class _BisectingKMeansParams(HasMaxIter, HasFeaturesCol, HasSeed, HasPredictionC "proportion of points (if < 1.0) of a divisible cluster.", typeConverter=TypeConverters.toFloat) - def __init__(self): - super(_BisectingKMeansParams, self).__init__() + def __init__(self, *args): + super(_BisectingKMeansParams, self).__init__(*args) self._setDefault(maxIter=20, k=4, minDivisibleClusterSize=1.0) @since("2.0.0") @@ -914,6 +918,8 @@ class BisectingKMeans(JavaEstimator, _BisectingKMeansParams, JavaMLWritable, Jav array([ True, True], dtype=bool) >>> model.clusterCenters()[1] == model2.clusterCenters()[1] array([ True, True], dtype=bool) + >>> model.transform(df).take(1) == model2.transform(df).take(1) + True .. versionadded:: 2.0.0 """ @@ -1072,8 +1078,8 @@ class _LDAParams(HasMaxIter, HasFeaturesCol, HasSeed, HasCheckpointInterval): " partition is lost, so set this bit with care.", TypeConverters.toBoolean) - def __init__(self): - super(_LDAParams, self).__init__() + def __init__(self, *args): + super(_LDAParams, self).__init__(*args) self._setDefault(maxIter=20, checkpointInterval=10, k=10, optimizer="online", learningOffset=1024.0, learningDecay=0.51, subsamplingRate=0.05, optimizeDocConcentration=True, @@ -1389,6 +1395,8 @@ class LDA(JavaEstimator, _LDAParams, JavaMLReadable, JavaMLWritable): >>> local_model_path = temp_path + "/lda_local_model" >>> localModel.save(local_model_path) >>> sameLocalModel = LocalLDAModel.load(local_model_path) + >>> model.transform(df).take(1) == sameLocalModel.transform(df).take(1) + True .. versionadded:: 2.0.0 """ @@ -1600,8 +1608,8 @@ class _PowerIterationClusteringParams(HasMaxIter, HasWeightCol): "Name of the input column for destination vertex IDs.", typeConverter=TypeConverters.toString) - def __init__(self): - super(_PowerIterationClusteringParams, self).__init__() + def __init__(self, *args): + super(_PowerIterationClusteringParams, self).__init__(*args) self._setDefault(k=2, maxIter=20, initMode="random", srcCol="src", dstCol="dst") @since("2.4.0") @@ -1677,6 +1685,8 @@ class PowerIterationClustering(_PowerIterationClusteringParams, JavaParams, Java 2 >>> pic2.getMaxIter() 40 + >>> pic2.assignClusters(df).take(6) == assignments.take(6) + True .. versionadded:: 2.4.0 """ diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index a319dace6869a..2220293d54ba4 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -92,6 +92,8 @@ class Binarizer(JavaTransformer, HasThreshold, HasThresholds, HasInputCol, HasOu >>> loadedBinarizer = Binarizer.load(binarizerPath) >>> loadedBinarizer.getThreshold() == binarizer.getThreshold() True + >>> loadedBinarizer.transform(df).take(1) == binarizer.transform(df).take(1) + True >>> df2 = spark.createDataFrame([(0.5, 0.3)], ["values1", "values2"]) >>> binarizer2 = Binarizer(thresholds=[0.0, 1.0]) >>> binarizer2.setInputCols(["values1", "values2"]).setOutputCols(["output1", "output2"]) @@ -195,6 +197,10 @@ class _LSHParams(HasInputCol, HasOutputCol): "and decreasing it improves the running performance.", typeConverter=TypeConverters.toInt) + def __init__(self, *args): + super(_LSHParams, self).__init__(*args) + self._setDefault(numHashTables=1) + def getNumHashTables(self): """ Gets the value of numHashTables or its default value. @@ -390,7 +396,6 @@ def __init__(self, inputCol=None, outputCol=None, seed=None, numHashTables=1, super(BucketedRandomProjectionLSH, self).__init__() self._java_obj = \ self._new_java_obj("org.apache.spark.ml.feature.BucketedRandomProjectionLSH", self.uid) - self._setDefault(numHashTables=1) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -477,6 +482,8 @@ class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, HasInputCols, HasOu >>> loadedBucketizer = Bucketizer.load(bucketizerPath) >>> loadedBucketizer.getSplits() == bucketizer.getSplits() True + >>> loadedBucketizer.transform(df).take(1) == bucketizer.transform(df).take(1) + True >>> bucketed = bucketizer.setHandleInvalid("skip").transform(df).collect() >>> len(bucketed) 4 @@ -733,6 +740,8 @@ class CountVectorizer(JavaEstimator, _CountVectorizerParams, JavaMLReadable, Jav >>> loadedModel = CountVectorizerModel.load(modelPath) >>> loadedModel.vocabulary == model.vocabulary True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True >>> fromVocabModel = CountVectorizerModel.from_vocabulary(["a", "b", "c"], ... inputCol="raw", outputCol="vectors") >>> fromVocabModel.transform(df).show(truncate=False) @@ -920,6 +929,8 @@ class DCT(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWrit >>> dctPath = temp_path + "/dct" >>> dct.save(dctPath) >>> loadedDtc = DCT.load(dctPath) + >>> loadedDtc.transform(df1).take(1) == dct.transform(df1).take(1) + True >>> loadedDtc.getInverse() False @@ -1003,6 +1014,8 @@ class ElementwiseProduct(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReada >>> loadedEp = ElementwiseProduct.load(elementwiseProductPath) >>> loadedEp.getScalingVec() == ep.getScalingVec() True + >>> loadedEp.transform(df).take(1) == ep.transform(df).take(1) + True .. versionadded:: 1.5.0 """ @@ -1201,6 +1214,8 @@ class HashingTF(JavaTransformer, HasInputCol, HasOutputCol, HasNumFeatures, Java >>> loadedHashingTF = HashingTF.load(hashingTFPath) >>> loadedHashingTF.getNumFeatures() == hashingTF.getNumFeatures() True + >>> loadedHashingTF.transform(df).take(1) == hashingTF.transform(df).take(1) + True >>> hashingTF.indexOf("b") 5 @@ -1292,6 +1307,10 @@ def getMinDocFreq(self): """ return self.getOrDefault(self.minDocFreq) + def __init__(self, *args): + super(_IDFParams, self).__init__(*args) + self._setDefault(minDocFreq=0) + @inherit_doc class IDF(JavaEstimator, _IDFParams, JavaMLReadable, JavaMLWritable): @@ -1345,7 +1364,6 @@ def __init__(self, minDocFreq=0, inputCol=None, outputCol=None): """ super(IDF, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.IDF", self.uid) - self._setDefault(minDocFreq=0) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -1445,6 +1463,10 @@ class _ImputerParams(HasInputCol, HasInputCols, HasOutputCol, HasOutputCols, Has "The placeholder for the missing values. All occurrences of missingValue " "will be imputed.", typeConverter=TypeConverters.toFloat) + def __init__(self, *args): + super(_ImputerParams, self).__init__(*args) + self._setDefault(strategy="mean", missingValue=float("nan"), relativeError=0.001) + @since("2.2.0") def getStrategy(self): """ @@ -1582,7 +1604,6 @@ def __init__(self, strategy="mean", missingValue=float("nan"), inputCols=None, """ super(Imputer, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.Imputer", self.uid) - self._setDefault(strategy="mean", missingValue=float("nan"), relativeError=0.001) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -1811,6 +1832,8 @@ class MaxAbsScaler(JavaEstimator, _MaxAbsScalerParams, JavaMLReadable, JavaMLWri >>> loadedModel = MaxAbsScalerModel.load(modelPath) >>> loadedModel.maxAbs == model.maxAbs True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True .. versionadded:: 2.0.0 """ @@ -1951,7 +1974,6 @@ def __init__(self, inputCol=None, outputCol=None, seed=None, numHashTables=1): """ super(MinHashLSH, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.MinHashLSH", self.uid) - self._setDefault(numHashTables=1) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -2002,6 +2024,10 @@ class _MinMaxScalerParams(HasInputCol, HasOutputCol): max = Param(Params._dummy(), "max", "Upper bound of the output feature range", typeConverter=TypeConverters.toFloat) + def __init__(self, *args): + super(_MinMaxScalerParams, self).__init__(*args) + self._setDefault(min=0.0, max=1.0) + @since("1.6.0") def getMin(self): """ @@ -2065,6 +2091,8 @@ class MinMaxScaler(JavaEstimator, _MinMaxScalerParams, JavaMLReadable, JavaMLWri True >>> loadedModel.originalMax == model.originalMax True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True .. versionadded:: 1.6.0 """ @@ -2076,7 +2104,6 @@ def __init__(self, min=0.0, max=1.0, inputCol=None, outputCol=None): """ super(MinMaxScaler, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.MinMaxScaler", self.uid) - self._setDefault(min=0.0, max=1.0) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -2209,6 +2236,8 @@ class NGram(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWr >>> loadedNGram = NGram.load(ngramPath) >>> loadedNGram.getN() == ngram.getN() True + >>> loadedNGram.transform(df).take(1) == ngram.transform(df).take(1) + True .. versionadded:: 1.5.0 """ @@ -2289,6 +2318,8 @@ class Normalizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, Jav >>> loadedNormalizer = Normalizer.load(normalizerPath) >>> loadedNormalizer.getP() == normalizer.getP() True + >>> loadedNormalizer.transform(df).take(1) == normalizer.transform(df).take(1) + True .. versionadded:: 1.4.0 """ @@ -2362,6 +2393,10 @@ class _OneHotEncoderParams(HasInputCol, HasInputCols, HasOutputCol, HasOutputCol dropLast = Param(Params._dummy(), "dropLast", "whether to drop the last category", typeConverter=TypeConverters.toBoolean) + def __init__(self, *args): + super(_OneHotEncoderParams, self).__init__(*args) + self._setDefault(handleInvalid="error", dropLast=True) + @since("2.3.0") def getDropLast(self): """ @@ -2422,6 +2457,8 @@ class OneHotEncoder(JavaEstimator, _OneHotEncoderParams, JavaMLReadable, JavaMLW >>> loadedModel = OneHotEncoderModel.load(modelPath) >>> loadedModel.categorySizes == model.categorySizes True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True .. versionadded:: 2.3.0 """ @@ -2436,7 +2473,6 @@ def __init__(self, inputCols=None, outputCols=None, handleInvalid="error", dropL super(OneHotEncoder, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.feature.OneHotEncoder", self.uid) - self._setDefault(handleInvalid="error", dropLast=True) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -2583,6 +2619,8 @@ class PolynomialExpansion(JavaTransformer, HasInputCol, HasOutputCol, JavaMLRead >>> loadedPx = PolynomialExpansion.load(polyExpansionPath) >>> loadedPx.getDegree() == px.getDegree() True + >>> loadedPx.transform(df).take(1) == px.transform(df).take(1) + True .. versionadded:: 1.4.0 """ @@ -2879,6 +2917,11 @@ class _RobustScalerParams(HasInputCol, HasOutputCol, HasRelativeError): withScaling = Param(Params._dummy(), "withScaling", "Whether to scale the data to " "quantile range", typeConverter=TypeConverters.toBoolean) + def __init__(self, *args): + super(_RobustScalerParams, self).__init__(*args) + self._setDefault(lower=0.25, upper=0.75, withCentering=False, withScaling=True, + relativeError=0.001) + @since("3.0.0") def getLower(self): """ @@ -2954,6 +2997,8 @@ class RobustScaler(JavaEstimator, _RobustScalerParams, JavaMLReadable, JavaMLWri True >>> loadedModel.range == model.range True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True .. versionadded:: 3.0.0 """ @@ -2967,8 +3012,6 @@ def __init__(self, lower=0.25, upper=0.75, withCentering=False, withScaling=True """ super(RobustScaler, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.RobustScaler", self.uid) - self._setDefault(lower=0.25, upper=0.75, withCentering=False, withScaling=True, - relativeError=0.001) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -3113,6 +3156,8 @@ class RegexTokenizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, True >>> loadedReTokenizer.getGaps() == reTokenizer.getGaps() True + >>> loadedReTokenizer.transform(df).take(1) == reTokenizer.transform(df).take(1) + True .. versionadded:: 1.4.0 """ @@ -3237,6 +3282,8 @@ class SQLTransformer(JavaTransformer, JavaMLReadable, JavaMLWritable): >>> loadedSqlTrans = SQLTransformer.load(sqlTransformerPath) >>> loadedSqlTrans.getStatement() == sqlTrans.getStatement() True + >>> loadedSqlTrans.transform(df).take(1) == sqlTrans.transform(df).take(1) + True .. versionadded:: 1.6.0 """ @@ -3291,6 +3338,10 @@ class _StandardScalerParams(HasInputCol, HasOutputCol): withStd = Param(Params._dummy(), "withStd", "Scale to unit standard deviation", typeConverter=TypeConverters.toBoolean) + def __init__(self, *args): + super(_StandardScalerParams, self).__init__(*args) + self._setDefault(withMean=False, withStd=True) + @since("1.4.0") def getWithMean(self): """ @@ -3348,6 +3399,8 @@ class StandardScaler(JavaEstimator, _StandardScalerParams, JavaMLReadable, JavaM True >>> loadedModel.mean == model.mean True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True .. versionadded:: 1.4.0 """ @@ -3359,7 +3412,6 @@ def __init__(self, withMean=False, withStd=True, inputCol=None, outputCol=None): """ super(StandardScaler, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.StandardScaler", self.uid) - self._setDefault(withMean=False, withStd=True) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -3513,6 +3565,8 @@ class StringIndexer(JavaEstimator, _StringIndexerParams, JavaMLReadable, JavaMLW >>> loadedInverter = IndexToString.load(indexToStringPath) >>> loadedInverter.getLabels() == inverter.getLabels() True + >>> loadedModel.transform(stringIndDf).take(1) == model.transform(stringIndDf).take(1) + True >>> stringIndexer.getStringOrderType() 'frequencyDesc' >>> stringIndexer = StringIndexer(inputCol="label", outputCol="indexed", handleInvalid="error", @@ -3798,6 +3852,8 @@ class StopWordsRemover(JavaTransformer, HasInputCol, HasOutputCol, HasInputCols, True >>> loadedRemover.getCaseSensitive() == remover.getCaseSensitive() True + >>> loadedRemover.transform(df).take(1) == remover.transform(df).take(1) + True >>> df2 = spark.createDataFrame([(["a", "b", "c"], ["a", "b"])], ["text1", "text2"]) >>> remover2 = StopWordsRemover(stopWords=["b"]) >>> remover2.setInputCols(["text1", "text2"]).setOutputCols(["words1", "words2"]) @@ -4109,6 +4165,10 @@ class _VectorIndexerParams(HasInputCol, HasOutputCol, HasHandleInvalid): "of categories of the feature).", typeConverter=TypeConverters.toString) + def __init__(self, *args): + super(_VectorIndexerParams, self).__init__(*args) + self._setDefault(maxCategories=20, handleInvalid="error") + @since("1.4.0") def getMaxCategories(self): """ @@ -4189,6 +4249,8 @@ class VectorIndexer(JavaEstimator, _VectorIndexerParams, JavaMLReadable, JavaMLW True >>> loadedModel.categoryMaps == model.categoryMaps True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True >>> dfWithInvalid = spark.createDataFrame([(Vectors.dense([3.0, 1.0]),)], ["a"]) >>> indexer.getHandleInvalid() 'error' @@ -4209,7 +4271,6 @@ def __init__(self, maxCategories=20, inputCol=None, outputCol=None, handleInvali """ super(VectorIndexer, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.VectorIndexer", self.uid) - self._setDefault(maxCategories=20, handleInvalid="error") kwargs = self._input_kwargs self.setParams(**kwargs) @@ -4332,6 +4393,8 @@ class VectorSlicer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, J True >>> loadedVs.getNames() == vs.getNames() True + >>> loadedVs.transform(df).take(1) == vs.transform(df).take(1) + True .. versionadded:: 1.6.0 """ @@ -4431,6 +4494,11 @@ class _Word2VecParams(HasStepSize, HasMaxIter, HasSeed, HasInputCol, HasOutputCo "be divided into chunks up to the size.", typeConverter=TypeConverters.toInt) + def __init__(self, *args): + super(_Word2VecParams, self).__init__(*args) + self._setDefault(vectorSize=100, minCount=5, numPartitions=1, stepSize=0.025, maxIter=1, + windowSize=5, maxSentenceLength=1000) + @since("1.4.0") def getVectorSize(self): """ @@ -4524,6 +4592,8 @@ class Word2Vec(JavaEstimator, _Word2VecParams, JavaMLReadable, JavaMLWritable): True >>> loadedModel.getVectors().first().vector == model.getVectors().first().vector True + >>> loadedModel.transform(doc).take(1) == model.transform(doc).take(1) + True .. versionadded:: 1.4.0 """ @@ -4537,8 +4607,6 @@ def __init__(self, vectorSize=100, minCount=5, numPartitions=1, stepSize=0.025, """ super(Word2Vec, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.Word2Vec", self.uid) - self._setDefault(vectorSize=100, minCount=5, numPartitions=1, stepSize=0.025, maxIter=1, - windowSize=5, maxSentenceLength=1000) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -4730,6 +4798,8 @@ class PCA(JavaEstimator, _PCAParams, JavaMLReadable, JavaMLWritable): True >>> loadedModel.explainedVariance == model.explainedVariance True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True .. versionadded:: 1.5.0 """ @@ -4846,6 +4916,11 @@ class _RFormulaParams(HasFeaturesCol, HasLabelCol, HasHandleInvalid): "additional bucket, at index numLabels).", typeConverter=TypeConverters.toString) + def __init__(self, *args): + super(_RFormulaParams, self).__init__(*args) + self._setDefault(forceIndexLabel=False, stringIndexerOrderType="frequencyDesc", + handleInvalid="error") + @since("1.5.0") def getFormula(self): """ @@ -4948,8 +5023,6 @@ def __init__(self, formula=None, featuresCol="features", labelCol="label", """ super(RFormula, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.RFormula", self.uid) - self._setDefault(forceIndexLabel=False, stringIndexerOrderType="frequencyDesc", - handleInvalid="error") kwargs = self._input_kwargs self.setParams(**kwargs) @@ -5058,6 +5131,11 @@ class _SelectorParams(HasFeaturesCol, HasOutputCol, HasLabelCol): fwe = Param(Params._dummy(), "fwe", "The upper bound of the expected family-wise error rate.", typeConverter=TypeConverters.toFloat) + def __init__(self, *args): + super(_SelectorParams, self).__init__(*args) + self._setDefault(numTopFeatures=50, selectorType="numTopFeatures", percentile=0.1, + fpr=0.05, fdr=0.05, fwe=0.05) + @since("2.1.0") def getSelectorType(self): """ @@ -5257,6 +5335,8 @@ class ANOVASelector(_Selector, JavaMLReadable, JavaMLWritable): >>> loadedModel = ANOVASelectorModel.load(modelPath) >>> loadedModel.selectedFeatures == model.selectedFeatures True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True .. versionadded:: 3.1.0 """ @@ -5272,8 +5352,6 @@ def __init__(self, numTopFeatures=50, featuresCol="features", outputCol=None, """ super(ANOVASelector, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.ANOVASelector", self.uid) - self._setDefault(numTopFeatures=50, selectorType="numTopFeatures", percentile=0.1, - fpr=0.05, fdr=0.05, fwe=0.05) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -5356,6 +5434,8 @@ class ChiSqSelector(_Selector, JavaMLReadable, JavaMLWritable): >>> loadedModel = ChiSqSelectorModel.load(modelPath) >>> loadedModel.selectedFeatures == model.selectedFeatures True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True .. versionadded:: 2.0.0 """ @@ -5371,8 +5451,6 @@ def __init__(self, numTopFeatures=50, featuresCol="features", outputCol=None, """ super(ChiSqSelector, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.ChiSqSelector", self.uid) - self._setDefault(numTopFeatures=50, selectorType="numTopFeatures", percentile=0.1, - fpr=0.05, fdr=0.05, fwe=0.05) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -5459,6 +5537,8 @@ class FValueSelector(_Selector, JavaMLReadable, JavaMLWritable): >>> loadedModel = FValueSelectorModel.load(modelPath) >>> loadedModel.selectedFeatures == model.selectedFeatures True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True .. versionadded:: 3.1.0 """ @@ -5474,8 +5554,6 @@ def __init__(self, numTopFeatures=50, featuresCol="features", outputCol=None, """ super(FValueSelector, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.FValueSelector", self.uid) - self._setDefault(numTopFeatures=50, selectorType="numTopFeatures", percentile=0.1, - fpr=0.05, fdr=0.05, fwe=0.05) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -5652,6 +5730,8 @@ class VarianceThresholdSelector(JavaEstimator, _VarianceThresholdSelectorParams, >>> loadedModel = VarianceThresholdSelectorModel.load(modelPath) >>> loadedModel.selectedFeatures == model.selectedFeatures True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True .. versionadded:: 3.1.0 """ diff --git a/python/pyspark/ml/fpm.py b/python/pyspark/ml/fpm.py index a1a8a4e3e3ac4..37d3b6eec02d0 100644 --- a/python/pyspark/ml/fpm.py +++ b/python/pyspark/ml/fpm.py @@ -55,8 +55,8 @@ class _FPGrowthParams(HasPredictionCol): "but will affect the association rules generation.", typeConverter=TypeConverters.toFloat) - def __init__(self): - super(_FPGrowthParams, self).__init__() + def __init__(self, *args): + super(_FPGrowthParams, self).__init__(*args) self._setDefault(minSupport=0.3, minConfidence=0.8, itemsCol="items", predictionCol="prediction") @@ -197,6 +197,11 @@ class FPGrowth(JavaEstimator, _FPGrowthParams, JavaMLWritable, JavaMLReadable): >>> new_data = spark.createDataFrame([(["t", "s"], )], ["items"]) >>> sorted(fpm.transform(new_data).first().newPrediction) ['x', 'y', 'z'] + >>> model_path = temp_path + "/fpm_model" + >>> fpm.save(model_path) + >>> model2 = FPGrowthModel.load(model_path) + >>> fpm.transform(data).take(1) == model2.transform(data).take(1) + True .. versionadded:: 2.2.0 """ diff --git a/python/pyspark/ml/recommendation.py b/python/pyspark/ml/recommendation.py index 99d80aa867bda..62b856046234a 100644 --- a/python/pyspark/ml/recommendation.py +++ b/python/pyspark/ml/recommendation.py @@ -46,6 +46,10 @@ class _ALSModelParams(HasPredictionCol, HasBlockSize): "Supported values: 'nan', 'drop'.", typeConverter=TypeConverters.toString) + def __init__(self, *args): + super(_ALSModelParams, self).__init__(*args) + self._setDefault(blockSize=4096) + @since("1.4.0") def getUserCol(self): """ @@ -99,6 +103,14 @@ class _ALSParams(_ALSModelParams, HasMaxIter, HasRegParam, HasCheckpointInterval "StorageLevel for ALS model factors.", typeConverter=TypeConverters.toString) + def __init__(self, *args): + super(_ALSParams, self).__init__(*args) + self._setDefault(rank=10, maxIter=10, regParam=0.1, numUserBlocks=10, numItemBlocks=10, + implicitPrefs=False, alpha=1.0, userCol="user", itemCol="item", + ratingCol="rating", nonnegative=False, checkpointInterval=10, + intermediateStorageLevel="MEMORY_AND_DISK", + finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan") + @since("1.4.0") def getRank(self): """ @@ -275,6 +287,8 @@ class ALS(JavaEstimator, _ALSParams, JavaMLWritable, JavaMLReadable): True >>> sorted(model.itemFactors.collect()) == sorted(model2.itemFactors.collect()) True + >>> model.transform(test).take(1) == model2.transform(test).take(1) + True .. versionadded:: 1.4.0 """ @@ -294,12 +308,6 @@ def __init__(self, rank=10, maxIter=10, regParam=0.1, numUserBlocks=10, numItemB """ super(ALS, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.recommendation.ALS", self.uid) - self._setDefault(rank=10, maxIter=10, regParam=0.1, numUserBlocks=10, numItemBlocks=10, - implicitPrefs=False, alpha=1.0, userCol="user", itemCol="item", - ratingCol="rating", nonnegative=False, checkpointInterval=10, - intermediateStorageLevel="MEMORY_AND_DISK", - finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan", - blockSize=4096) kwargs = self._input_kwargs self.setParams(**kwargs) diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 6fe6486c5a04a..4a8d1530b8a6f 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -104,8 +104,8 @@ class _LinearRegressionParams(_PredictorParams, HasRegParam, HasElasticNetParam, "robustness. Must be > 1.0. Only valid when loss is huber", typeConverter=TypeConverters.toFloat) - def __init__(self): - super(_LinearRegressionParams, self).__init__() + def __init__(self, *args): + super(_LinearRegressionParams, self).__init__(*args) self._setDefault(maxIter=100, regParam=0.0, tol=1e-6, loss="squaredError", epsilon=1.35, blockSize=1) @@ -190,6 +190,8 @@ class LinearRegression(_JavaRegressor, _LinearRegressionParams, JavaMLWritable, True >>> model.intercept == model2.intercept True + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True >>> model.numFeatures 1 >>> model.write().format("pmml").save(model_path + "_2") @@ -622,8 +624,8 @@ class _IsotonicRegressionParams(HasFeaturesCol, HasLabelCol, HasPredictionCol, H "The index of the feature if featuresCol is a vector column, no effect otherwise.", typeConverter=TypeConverters.toInt) - def __init__(self): - super(_IsotonicRegressionParams, self).__init__() + def __init__(self, *args): + super(_IsotonicRegressionParams, self).__init__(*args) self._setDefault(isotonic=True, featureIndex=0) def getIsotonic(self): @@ -675,6 +677,8 @@ class IsotonicRegression(JavaEstimator, _IsotonicRegressionParams, HasWeightCol, True >>> model.predictions == model2.predictions True + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True .. versionadded:: 1.6.0 """ @@ -814,8 +818,8 @@ class _DecisionTreeRegressorParams(_DecisionTreeParams, _TreeRegressorParams, Ha .. versionadded:: 3.0.0 """ - def __init__(self): - super(_DecisionTreeRegressorParams, self).__init__() + def __init__(self, *args): + super(_DecisionTreeRegressorParams, self).__init__(*args) self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="variance", leafCol="", minWeightFractionPerNode=0.0) @@ -876,7 +880,8 @@ class DecisionTreeRegressor(_JavaRegressor, _DecisionTreeRegressorParams, JavaML True >>> model.transform(test1).head().variance 0.0 - + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True >>> df3 = spark.createDataFrame([ ... (1.0, 0.2, Vectors.dense(1.0)), ... (1.0, 0.8, Vectors.dense(1.0)), @@ -1060,8 +1065,8 @@ class _RandomForestRegressorParams(_RandomForestParams, _TreeRegressorParams): .. versionadded:: 3.0.0 """ - def __init__(self): - super(_RandomForestRegressorParams, self).__init__() + def __init__(self, *args): + super(_RandomForestRegressorParams, self).__init__(*args) self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="variance", subsamplingRate=1.0, numTrees=20, @@ -1127,6 +1132,8 @@ class RandomForestRegressor(_JavaRegressor, _RandomForestRegressorParams, JavaML >>> model2 = RandomForestRegressionModel.load(model_path) >>> model.featureImportances == model2.featureImportances True + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True .. versionadded:: 1.4.0 """ @@ -1319,8 +1326,8 @@ class _GBTRegressorParams(_GBTParams, _TreeRegressorParams): "Supported options: " + ", ".join(supportedLossTypes), typeConverter=TypeConverters.toString) - def __init__(self): - super(_GBTRegressorParams, self).__init__() + def __init__(self, *args): + super(_GBTRegressorParams, self).__init__(*args) self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, maxMemoryInMB=256, cacheNodeIds=False, subsamplingRate=1.0, checkpointInterval=10, lossType="squared", maxIter=20, stepSize=0.1, @@ -1390,6 +1397,8 @@ class GBTRegressor(_JavaRegressor, _GBTRegressorParams, JavaMLWritable, JavaMLRe True >>> model.treeWeights == model2.treeWeights True + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True >>> model.trees [DecisionTreeRegressionModel...depth=..., DecisionTreeRegressionModel...] >>> validation = spark.createDataFrame([(0.0, Vectors.dense(-1.0))], @@ -1642,8 +1651,8 @@ class _AFTSurvivalRegressionParams(_PredictorParams, HasMaxIter, HasTol, HasFitI "corresponding quantileProbabilities if it is set.", typeConverter=TypeConverters.toString) - def __init__(self): - super(_AFTSurvivalRegressionParams, self).__init__() + def __init__(self, *args): + super(_AFTSurvivalRegressionParams, self).__init__(*args) self._setDefault(censorCol="censor", quantileProbabilities=[0.01, 0.05, 0.1, 0.25, 0.5, 0.75, 0.9, 0.95, 0.99], maxIter=100, tol=1E-6, blockSize=1) @@ -1722,6 +1731,8 @@ class AFTSurvivalRegression(_JavaRegressor, _AFTSurvivalRegressionParams, True >>> model.scale == model2.scale True + >>> model.transform(df).take(1) == model2.transform(df).take(1) + True .. versionadded:: 1.6.0 """ @@ -1906,8 +1917,8 @@ class _GeneralizedLinearRegressionParams(_PredictorParams, HasFitIntercept, HasM "or empty, we treat all instance offsets as 0.0", typeConverter=TypeConverters.toString) - def __init__(self): - super(_GeneralizedLinearRegressionParams, self).__init__() + def __init__(self, *args): + super(_GeneralizedLinearRegressionParams, self).__init__(*args) self._setDefault(family="gaussian", maxIter=25, tol=1e-6, regParam=0.0, solver="irls", variancePower=0.0, aggregationDepth=2) @@ -2025,6 +2036,8 @@ class GeneralizedLinearRegression(_JavaRegressor, _GeneralizedLinearRegressionPa True >>> model.coefficients[0] == model2.coefficients[0] True + >>> model.transform(df).take(1) == model2.transform(df).take(1) + True .. versionadded:: 2.0.0 """ @@ -2391,7 +2404,7 @@ def __repr__(self): class _FactorizationMachinesParams(_PredictorParams, HasMaxIter, HasStepSize, HasTol, - HasSolver, HasSeed, HasFitIntercept, HasRegParam): + HasSolver, HasSeed, HasFitIntercept, HasRegParam, HasWeightCol): """ Params for :py:class:`FMRegressor`, :py:class:`FMRegressionModel`, :py:class:`FMClassifier` and :py:class:`FMClassifierModel`. @@ -2416,8 +2429,8 @@ class _FactorizationMachinesParams(_PredictorParams, HasMaxIter, HasStepSize, Ha solver = Param(Params._dummy(), "solver", "The solver algorithm for optimization. Supported " + "options: gd, adamW. (Default adamW)", typeConverter=TypeConverters.toString) - def __init__(self): - super(_FactorizationMachinesParams, self).__init__() + def __init__(self, *args): + super(_FactorizationMachinesParams, self).__init__(*args) self._setDefault(factorSize=8, fitIntercept=True, fitLinear=True, regParam=0.0, miniBatchFraction=1.0, initStd=0.01, maxIter=100, stepSize=1.0, tol=1e-6, solver="adamW") @@ -2495,6 +2508,17 @@ class FMRegressor(_JavaRegressor, _FactorizationMachinesParams, JavaMLWritable, DenseVector([0.9978]) >>> model.factors DenseMatrix(1, 2, [0.0173, 0.0021], 1) + >>> model_path = temp_path + "/fm_model" + >>> model.save(model_path) + >>> model2 = FMRegressionModel.load(model_path) + >>> model2.intercept + -0.0032501766849261557 + >>> model2.linear + DenseVector([0.9978]) + >>> model2.factors + DenseMatrix(1, 2, [0.0173, 0.0021], 1) + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True .. versionadded:: 3.0.0 """ diff --git a/python/pyspark/ml/stat.py b/python/pyspark/ml/stat.py index 70de8425613ec..517c984252768 100644 --- a/python/pyspark/ml/stat.py +++ b/python/pyspark/ml/stat.py @@ -222,14 +222,14 @@ class Summarizer(object): +-----------------------------------+ |aggregate_metrics(features, weight)| +-----------------------------------+ - |[[1.0,1.0,1.0], 1] | + |{[1.0,1.0,1.0], 1} | +-----------------------------------+ >>> df.select(summarizer.summary(df.features)).show(truncate=False) +--------------------------------+ |aggregate_metrics(features, 1.0)| +--------------------------------+ - |[[1.0,1.5,2.0], 2] | + |{[1.0,1.5,2.0], 2} | +--------------------------------+ >>> df.select(Summarizer.mean(df.features, df.weight)).show(truncate=False) diff --git a/python/pyspark/ml/tests/test_param.py b/python/pyspark/ml/tests/test_param.py index e1abd59a2d7b2..44731568b6d1f 100644 --- a/python/pyspark/ml/tests/test_param.py +++ b/python/pyspark/ml/tests/test_param.py @@ -359,16 +359,14 @@ def test_java_params(self): and issubclass(cls, JavaParams) and not inspect.isabstract(cls) \ and not re.match("_?Java", name) and name != '_LSH' \ and name != '_Selector': - # NOTE: disable check_params_exist until there is parity with Scala API - - check_params(self, cls(), check_params_exist=False) + check_params(self, cls(), check_params_exist=True) # Additional classes that need explicit construction from pyspark.ml.feature import CountVectorizerModel, StringIndexerModel check_params(self, CountVectorizerModel.from_vocabulary(['a'], 'input'), - check_params_exist=False) + check_params_exist=True) check_params(self, StringIndexerModel.from_labels(['a', 'b'], 'input'), - check_params_exist=False) + check_params_exist=True) if __name__ == "__main__": diff --git a/python/pyspark/ml/tuning.py b/python/pyspark/ml/tuning.py index 7f3d942e2e456..d7800e0c9020e 100644 --- a/python/pyspark/ml/tuning.py +++ b/python/pyspark/ml/tuning.py @@ -206,6 +206,10 @@ class _CrossValidatorParams(_ValidatorParams): "with range [0, numFolds) and Spark will throw exception on out-of-range " + "fold numbers.", typeConverter=TypeConverters.toString) + def __init__(self, *args): + super(_CrossValidatorParams, self).__init__(*args) + self._setDefault(numFolds=3, foldCol="") + @since("1.4.0") def getNumFolds(self): """ @@ -262,6 +266,8 @@ class CrossValidator(Estimator, _CrossValidatorParams, HasParallelism, HasCollec [0.5, ... >>> evaluator.evaluate(cvModel.transform(dataset)) 0.8333... + >>> evaluator.evaluate(cvModelRead.transform(dataset)) + 0.8333... .. versionadded:: 1.4.0 """ @@ -274,7 +280,7 @@ def __init__(self, estimator=None, estimatorParamMaps=None, evaluator=None, numF seed=None, parallelism=1, collectSubModels=False, foldCol="") """ super(CrossValidator, self).__init__() - self._setDefault(numFolds=3, parallelism=1, foldCol="") + self._setDefault(parallelism=1) kwargs = self._input_kwargs self._set(**kwargs) @@ -600,6 +606,10 @@ class _TrainValidationSplitParams(_ValidatorParams): trainRatio = Param(Params._dummy(), "trainRatio", "Param for ratio between train and\ validation data. Must be between 0 and 1.", typeConverter=TypeConverters.toFloat) + def __init__(self, *args): + super(_TrainValidationSplitParams, self).__init__(*args) + self._setDefault(trainRatio=0.75) + @since("2.0.0") def getTrainRatio(self): """ @@ -645,8 +655,11 @@ class TrainValidationSplit(Estimator, _TrainValidationSplitParams, HasParallelis [0.5, ... >>> evaluator.evaluate(tvsModel.transform(dataset)) 0.833... + >>> evaluator.evaluate(tvsModelRead.transform(dataset)) + 0.833... .. versionadded:: 2.0.0 + """ @keyword_only @@ -657,7 +670,7 @@ def __init__(self, estimator=None, estimatorParamMaps=None, evaluator=None, trai parallelism=1, collectSubModels=False, seed=None) """ super(TrainValidationSplit, self).__init__() - self._setDefault(trainRatio=0.75, parallelism=1) + self._setDefault(parallelism=1) kwargs = self._input_kwargs self._set(**kwargs) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 02a6f0022e7ab..308642b136f75 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1996,7 +1996,7 @@ def map_from_arrays(col1, col2): +----------------+ | map| +----------------+ - |[2 -> a, 5 -> b]| + |{2 -> a, 5 -> b}| +----------------+ """ sc = SparkContext._active_spark_context @@ -2316,9 +2316,9 @@ def explode_outer(col): +---+----------+----+ | id| a_map| col| +---+----------+----+ - | 1|[x -> 1.0]| foo| - | 1|[x -> 1.0]| bar| - | 2| []|null| + | 1|{x -> 1.0}| foo| + | 1|{x -> 1.0}| bar| + | 2| {}|null| | 3| null|null| +---+----------+----+ """ @@ -2351,9 +2351,9 @@ def posexplode_outer(col): +---+----------+----+----+ | id| a_map| pos| col| +---+----------+----+----+ - | 1|[x -> 1.0]| 0| foo| - | 1|[x -> 1.0]| 1| bar| - | 2| []|null|null| + | 1|{x -> 1.0}| 0| foo| + | 1|{x -> 1.0}| 1| bar| + | 2| {}|null|null| | 3| null|null|null| +---+----------+----+----+ """ @@ -2750,7 +2750,7 @@ def map_entries(col): +----------------+ | entries| +----------------+ - |[[1, a], [2, b]]| + |[{1, a}, {2, b}]| +----------------+ """ sc = SparkContext._active_spark_context @@ -2770,7 +2770,7 @@ def map_from_entries(col): +----------------+ | map| +----------------+ - |[1 -> a, 2 -> b]| + |{1 -> a, 2 -> b}| +----------------+ """ sc = SparkContext._active_spark_context @@ -2822,7 +2822,7 @@ def map_concat(*cols): +------------------------+ |map3 | +------------------------+ - |[1 -> a, 2 -> b, 3 -> c]| + |{1 -> a, 2 -> b, 3 -> c}| +------------------------+ """ sc = SparkContext._active_spark_context @@ -3241,7 +3241,7 @@ def transform_keys(col, f): +-------------------------+ |data_upper | +-------------------------+ - |[BAR -> 2.0, FOO -> -2.0]| + |{BAR -> 2.0, FOO -> -2.0}| +-------------------------+ """ return _invoke_higher_order_function("TransformKeys", [col], [f]) @@ -3268,7 +3268,7 @@ def transform_values(col, f): +---------------------------------------+ |new_data | +---------------------------------------+ - |[OPS -> 34.0, IT -> 20.0, SALES -> 2.0]| + |{OPS -> 34.0, IT -> 20.0, SALES -> 2.0}| +---------------------------------------+ """ return _invoke_higher_order_function("TransformValues", [col], [f]) @@ -3294,7 +3294,7 @@ def map_filter(col, f): +--------------------------+ |data_filtered | +--------------------------+ - |[baz -> 32.0, foo -> 42.0]| + |{baz -> 32.0, foo -> 42.0}| +--------------------------+ """ return _invoke_higher_order_function("MapFilter", [col], [f]) @@ -3324,7 +3324,7 @@ def map_zip_with(col1, col2, f): +---------------------------+ |updated_data | +---------------------------+ - |[SALES -> 16.8, IT -> 48.0]| + |{SALES -> 16.8, IT -> 48.0}| +---------------------------+ """ return _invoke_higher_order_function("MapZipWith", [col1, col2], [f]) diff --git a/python/pyspark/tests/test_context.py b/python/pyspark/tests/test_context.py index 64fe3837e7697..f398cec344725 100644 --- a/python/pyspark/tests/test_context.py +++ b/python/pyspark/tests/test_context.py @@ -279,7 +279,7 @@ def test_allow_to_create_spark_context_in_executors(self): # SPARK-32160: SparkContext can be created in executors if the config is set. def create_spark_context(): - conf = SparkConf().set("spark.driver.allowSparkContextInExecutors", "true") + conf = SparkConf().set("spark.executor.allowSparkContext", "true") with SparkContext(conf=conf): pass diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 39f94651a0cb5..6fce7819897a6 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -246,11 +246,17 @@ statement | SET TIME ZONE interval #setTimeZone | SET TIME ZONE timezone=(STRING | LOCAL) #setTimeZone | SET TIME ZONE .*? #setTimeZone + | SET configKey (EQ .*?)? #setQuotedConfiguration | SET .*? #setConfiguration + | RESET configKey #resetQuotedConfiguration | RESET .*? #resetConfiguration | unsupportedHiveNativeCommands .*? #failNativeCommand ; +configKey + : quotedIdentifier + ; + unsupportedHiveNativeCommands : kw1=CREATE kw2=ROLE | kw1=DROP kw2=ROLE diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index e58b0ae64784d..477863a1b86d0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1973,15 +1973,9 @@ class Analyzer( } // We get an aggregate function, we need to wrap it in an AggregateExpression. case agg: AggregateFunction => - // TODO: SPARK-30276 Support Filter expression allows simultaneous use of DISTINCT - if (filter.isDefined) { - if (isDistinct) { - failAnalysis("DISTINCT and FILTER cannot be used in aggregate functions " + - "at the same time") - } else if (!filter.get.deterministic) { - failAnalysis("FILTER expression is non-deterministic, " + - "it cannot be used in aggregate functions") - } + if (filter.isDefined && !filter.get.deterministic) { + failAnalysis("FILTER expression is non-deterministic, " + + "it cannot be used in aggregate functions") } AggregateExpression(agg, Complete, isDistinct, filter) // This function is not an aggregate function, just return the resolved one. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 3989df5d29467..7e73667e4b85f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -354,6 +354,7 @@ object FunctionRegistry { expression[StringLocate]("position", true), expression[FormatString]("printf", true), expression[RegExpExtract]("regexp_extract"), + expression[RegExpExtractAll]("regexp_extract_all"), expression[RegExpReplace]("regexp_replace"), expression[StringRepeat]("repeat"), expression[StringReplace]("replace"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index ec0c34d4c7960..6fba3156c3919 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -219,12 +219,20 @@ class SessionCatalog( "you cannot create a database with this name.") } validateName(dbName) - val qualifiedPath = makeQualifiedPath(dbDefinition.locationUri) externalCatalog.createDatabase( - dbDefinition.copy(name = dbName, locationUri = qualifiedPath), + dbDefinition.copy(name = dbName, locationUri = makeQualifiedDBPath(dbDefinition.locationUri)), ignoreIfExists) } + private def makeQualifiedDBPath(locationUri: URI): URI = { + if (locationUri.isAbsolute) { + locationUri + } else { + val fullPath = new Path(conf.warehousePath, CatalogUtils.URIToString(locationUri)) + makeQualifiedPath(fullPath.toUri) + } + } + def dropDatabase(db: String, ignoreIfNotExists: Boolean, cascade: Boolean): Unit = { val dbName = formatDatabaseName(db) if (dbName == DEFAULT_DATABASE) { @@ -241,7 +249,8 @@ class SessionCatalog( def alterDatabase(dbDefinition: CatalogDatabase): Unit = { val dbName = formatDatabaseName(dbDefinition.name) requireDbExists(dbName) - externalCatalog.alterDatabase(dbDefinition.copy(name = dbName)) + externalCatalog.alterDatabase(dbDefinition.copy( + name = dbName, locationUri = makeQualifiedDBPath(dbDefinition.locationUri))) } def getDatabaseMetadata(db: String): CatalogDatabase = { @@ -283,8 +292,7 @@ class SessionCatalog( * by users. */ def getDefaultDBPath(db: String): URI = { - val database = formatDatabaseName(db) - new Path(new Path(conf.warehousePath), database + ".db").toUri + CatalogUtils.stringToURI(formatDatabaseName(db) + ".db") } // ---------------------------------------------------------------------------- @@ -317,7 +325,7 @@ class SessionCatalog( && !tableDefinition.storage.locationUri.get.isAbsolute) { // make the location of the table qualified. val qualifiedTableLocation = - makeQualifiedPath(tableDefinition.storage.locationUri.get) + makeQualifiedTablePath(tableDefinition.storage.locationUri.get, db) tableDefinition.copy( storage = tableDefinition.storage.copy(locationUri = Some(qualifiedTableLocation)), identifier = tableIdentifier) @@ -350,6 +358,16 @@ class SessionCatalog( } } + private def makeQualifiedTablePath(locationUri: URI, database: String): URI = { + if (locationUri.isAbsolute) { + locationUri + } else { + val dbName = formatDatabaseName(database) + val dbLocation = makeQualifiedDBPath(getDatabaseMetadata(dbName).locationUri) + new Path(new Path(dbLocation), CatalogUtils.URIToString(locationUri)).toUri + } + } + /** * Alter the metadata of an existing metastore table identified by `tableDefinition`. * @@ -369,7 +387,7 @@ class SessionCatalog( && !tableDefinition.storage.locationUri.get.isAbsolute) { // make the location of the table qualified. val qualifiedTableLocation = - makeQualifiedPath(tableDefinition.storage.locationUri.get) + makeQualifiedTablePath(tableDefinition.storage.locationUri.get, db) tableDefinition.copy( storage = tableDefinition.storage.copy(locationUri = Some(qualifiedTableLocation)), identifier = tableIdentifier) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index e27c021556377..e93dc588819b0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -297,6 +297,10 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit private lazy val dateFormatter = DateFormatter(zoneId) private lazy val timestampFormatter = TimestampFormatter.getFractionFormatter(zoneId) + // The brackets that are used in casting structs and maps to strings + private val (leftBracket, rightBracket) = + if (SQLConf.get.getConf(SQLConf.LEGACY_COMPLEX_TYPES_TO_STRING)) ("[", "]") else ("{", "}") + // UDFToString private[this] def castToString(from: DataType): Any => Any = from match { case CalendarIntervalType => @@ -330,7 +334,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit case MapType(kt, vt, _) => buildCast[MapData](_, map => { val builder = new UTF8StringBuilder - builder.append("[") + builder.append(leftBracket) if (map.numElements > 0) { val keyArray = map.keyArray() val valueArray = map.valueArray() @@ -355,13 +359,13 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit i += 1 } } - builder.append("]") + builder.append(rightBracket) builder.build() }) case StructType(fields) => buildCast[InternalRow](_, row => { val builder = new UTF8StringBuilder - builder.append("[") + builder.append(leftBracket) if (row.numFields > 0) { val st = fields.map(_.dataType) val toUTF8StringFuncs = st.map(castToString) @@ -378,7 +382,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit i += 1 } } - builder.append("]") + builder.append(rightBracket) builder.build() }) case pudt: PythonUserDefinedType => castToString(pudt.sqlType) @@ -962,7 +966,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit val getMapKeyArray = CodeGenerator.getValue(mapKeyArray, kt, loopIndex) val getMapValueArray = CodeGenerator.getValue(mapValueArray, vt, loopIndex) code""" - |$buffer.append("["); + |$buffer.append("$leftBracket"); |if ($map.numElements() > 0) { | $buffer.append($keyToStringFunc($getMapFirstKey)); | $buffer.append(" ->"); @@ -980,7 +984,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit | } | } |} - |$buffer.append("]"); + |$buffer.append("$rightBracket"); """.stripMargin } @@ -1015,9 +1019,9 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit (classOf[UTF8StringBuilder].getName, buffer.code) :: Nil) code""" - |$buffer.append("["); + |$buffer.append("$leftBracket"); |$writeStructCode - |$buffer.append("]"); + |$buffer.append("$rightBracket"); """.stripMargin } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala index 1af1636e1df75..8eb7f463e049c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala @@ -18,7 +18,9 @@ package org.apache.spark.sql.catalyst.expressions import java.util.Locale -import java.util.regex.{MatchResult, Pattern} +import java.util.regex.{Matcher, MatchResult, Pattern} + +import scala.collection.mutable.ArrayBuffer import org.apache.commons.text.StringEscapeUtils @@ -410,7 +412,7 @@ case class RegExpReplace(subject: Expression, regexp: Expression, rep: Expressio } } -object RegExpExtract { +object RegExpExtractBase { def checkGroupIndex(groupCount: Int, groupIndex: Int): Unit = { if (groupIndex < 0) { throw new IllegalArgumentException("The specified group index cannot be less than zero") @@ -421,20 +423,58 @@ object RegExpExtract { } } +abstract class RegExpExtractBase + extends TernaryExpression with ImplicitCastInputTypes with NullIntolerant { + def subject: Expression + def regexp: Expression + def idx: Expression + + // last regex in string, we will update the pattern iff regexp value changed. + @transient private var lastRegex: UTF8String = _ + // last regex pattern, we cache it for performance concern + @transient private var pattern: Pattern = _ + + override def inputTypes: Seq[AbstractDataType] = Seq(StringType, StringType, IntegerType) + override def children: Seq[Expression] = subject :: regexp :: idx :: Nil + + protected def getLastMatcher(s: Any, p: Any): Matcher = { + if (p != lastRegex) { + // regex value changed + lastRegex = p.asInstanceOf[UTF8String].clone() + pattern = Pattern.compile(lastRegex.toString) + } + pattern.matcher(s.toString) + } +} + /** * Extract a specific(idx) group identified by a Java regex. * * NOTE: this expression is not THREAD-SAFE, as it has some internal mutable status. */ @ExpressionDescription( - usage = "_FUNC_(str, regexp[, idx]) - Extracts a group that matches `regexp`.", + usage = """ + _FUNC_(str, regexp[, idx]) - Extract the first string in the `str` that match the `regexp` + expression and corresponding to the regex group index. + """, arguments = """ Arguments: * str - a string expression. - * regexp - a string representing a regular expression. - The regex string should be a Java regular expression. - * idx - an integer expression that representing the group index. The group index should be - non-negative. If `idx` is not specified, the default group index value is 1. + * regexp - a string representing a regular expression. The regex string should be a + Java regular expression. + + Since Spark 2.0, string literals (including regex patterns) are unescaped in our SQL + parser. For example, to match "\abc", a regular expression for `regexp` can be + "^\\abc$". + + There is a SQL config 'spark.sql.parser.escapedStringLiterals' that can be used to + fallback to the Spark 1.6 behavior regarding string literal parsing. For example, + if the config is enabled, the `regexp` that can match "\abc" is "^\abc$". + * idx - an integer expression that representing the group index. The regex maybe contains + multiple groups. `idx` indicates which regex group to extract. The group index should + be non-negative. The minimum value of `idx` is 0, which means matching the entire + regular expression. If `idx` is not specified, the default group index value is 1. The + `idx` parameter is the Java regex Matcher group() method index. """, examples = """ Examples: @@ -443,27 +483,17 @@ object RegExpExtract { """, since = "1.5.0") case class RegExpExtract(subject: Expression, regexp: Expression, idx: Expression) - extends TernaryExpression with ImplicitCastInputTypes with NullIntolerant { + extends RegExpExtractBase { def this(s: Expression, r: Expression) = this(s, r, Literal(1)) - // last regex in string, we will update the pattern iff regexp value changed. - @transient private var lastRegex: UTF8String = _ - // last regex pattern, we cache it for performance concern - @transient private var pattern: Pattern = _ - override def nullSafeEval(s: Any, p: Any, r: Any): Any = { - if (!p.equals(lastRegex)) { - // regex value changed - lastRegex = p.asInstanceOf[UTF8String].clone() - pattern = Pattern.compile(lastRegex.toString) - } - val m = pattern.matcher(s.toString) + val m = getLastMatcher(s, p) if (m.find) { val mr: MatchResult = m.toMatchResult val index = r.asInstanceOf[Int] - RegExpExtract.checkGroupIndex(mr.groupCount, index) + RegExpExtractBase.checkGroupIndex(mr.groupCount, index) val group = mr.group(index) - if (group == null) { // Pattern matched, but not optional group + if (group == null) { // Pattern matched, but it's an optional group UTF8String.EMPTY_UTF8 } else { UTF8String.fromString(group) @@ -474,13 +504,11 @@ case class RegExpExtract(subject: Expression, regexp: Expression, idx: Expressio } override def dataType: DataType = StringType - override def inputTypes: Seq[AbstractDataType] = Seq(StringType, StringType, IntegerType) - override def children: Seq[Expression] = subject :: regexp :: idx :: Nil override def prettyName: String = "regexp_extract" override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val classNamePattern = classOf[Pattern].getCanonicalName - val classNameRegExpExtract = classOf[RegExpExtract].getCanonicalName + val classNameRegExpExtractBase = classOf[RegExpExtractBase].getCanonicalName val matcher = ctx.freshName("matcher") val matchResult = ctx.freshName("matchResult") @@ -504,7 +532,7 @@ case class RegExpExtract(subject: Expression, regexp: Expression, idx: Expressio $termPattern.matcher($subject.toString()); if ($matcher.find()) { java.util.regex.MatchResult $matchResult = $matcher.toMatchResult(); - $classNameRegExpExtract.checkGroupIndex($matchResult.groupCount(), $idx); + $classNameRegExpExtractBase.checkGroupIndex($matchResult.groupCount(), $idx); if ($matchResult.group($idx) == null) { ${ev.value} = UTF8String.EMPTY_UTF8; } else { @@ -518,3 +546,105 @@ case class RegExpExtract(subject: Expression, regexp: Expression, idx: Expressio }) } } + +/** + * Extract all specific(idx) groups identified by a Java regex. + * + * NOTE: this expression is not THREAD-SAFE, as it has some internal mutable status. + */ +@ExpressionDescription( + usage = """ + _FUNC_(str, regexp[, idx]) - Extract all strings in the `str` that match the `regexp` + expression and corresponding to the regex group index. + """, + arguments = """ + Arguments: + * str - a string expression. + * regexp - a string representing a regular expression. The regex string should be a + Java regular expression. + + Since Spark 2.0, string literals (including regex patterns) are unescaped in our SQL + parser. For example, to match "\abc", a regular expression for `regexp` can be + "^\\abc$". + + There is a SQL config 'spark.sql.parser.escapedStringLiterals' that can be used to + fallback to the Spark 1.6 behavior regarding string literal parsing. For example, + if the config is enabled, the `regexp` that can match "\abc" is "^\abc$". + * idx - an integer expression that representing the group index. The regex may contains + multiple groups. `idx` indicates which regex group to extract. The group index should + be non-negative. The minimum value of `idx` is 0, which means matching the entire + regular expression. If `idx` is not specified, the default group index value is 1. The + `idx` parameter is the Java regex Matcher group() method index. + """, + examples = """ + Examples: + > SELECT _FUNC_('100-200, 300-400', '(\\d+)-(\\d+)', 1); + ["100","300"] + """, + since = "3.1.0") +case class RegExpExtractAll(subject: Expression, regexp: Expression, idx: Expression) + extends RegExpExtractBase { + def this(s: Expression, r: Expression) = this(s, r, Literal(1)) + + override def nullSafeEval(s: Any, p: Any, r: Any): Any = { + val m = getLastMatcher(s, p) + val matchResults = new ArrayBuffer[UTF8String]() + while(m.find) { + val mr: MatchResult = m.toMatchResult + val index = r.asInstanceOf[Int] + RegExpExtractBase.checkGroupIndex(mr.groupCount, index) + val group = mr.group(index) + if (group == null) { // Pattern matched, but it's an optional group + matchResults += UTF8String.EMPTY_UTF8 + } else { + matchResults += UTF8String.fromString(group) + } + } + + new GenericArrayData(matchResults.toArray.asInstanceOf[Array[Any]]) + } + + override def dataType: DataType = ArrayType(StringType) + override def prettyName: String = "regexp_extract_all" + + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val classNamePattern = classOf[Pattern].getCanonicalName + val classNameRegExpExtractBase = classOf[RegExpExtractBase].getCanonicalName + val arrayClass = classOf[GenericArrayData].getName + val matcher = ctx.freshName("matcher") + val matchResult = ctx.freshName("matchResult") + val matchResults = ctx.freshName("matchResults") + + val termLastRegex = ctx.addMutableState("UTF8String", "lastRegex") + val termPattern = ctx.addMutableState(classNamePattern, "pattern") + + val setEvNotNull = if (nullable) { + s"${ev.isNull} = false;" + } else { + "" + } + nullSafeCodeGen(ctx, ev, (subject, regexp, idx) => { + s""" + | if (!$regexp.equals($termLastRegex)) { + | // regex value changed + | $termLastRegex = $regexp.clone(); + | $termPattern = $classNamePattern.compile($termLastRegex.toString()); + | } + | java.util.regex.Matcher $matcher = $termPattern.matcher($subject.toString()); + | java.util.ArrayList $matchResults = new java.util.ArrayList(); + | while ($matcher.find()) { + | java.util.regex.MatchResult $matchResult = $matcher.toMatchResult(); + | $classNameRegExpExtractBase.checkGroupIndex($matchResult.groupCount(), $idx); + | if ($matchResult.group($idx) == null) { + | $matchResults.add(UTF8String.EMPTY_UTF8); + | } else { + | $matchResults.add(UTF8String.fromString($matchResult.group($idx))); + | } + | } + | ${ev.value} = + | new $arrayClass($matchResults.toArray(new UTF8String[$matchResults.size()])); + | $setEvNotNull + """ + }) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala index 15aa02ff677de..af3a8fe684bb3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, AggregateFunction, Complete} +import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Expand, LogicalPlan} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.types.IntegerType @@ -81,10 +81,10 @@ import org.apache.spark.sql.types.IntegerType * COUNT(DISTINCT cat1) as cat1_cnt, * COUNT(DISTINCT cat2) as cat2_cnt, * SUM(value) FILTER (WHERE id > 1) AS total - * FROM - * data - * GROUP BY - * key + * FROM + * data + * GROUP BY + * key * }}} * * This translates to the following (pseudo) logical plan: @@ -93,7 +93,7 @@ import org.apache.spark.sql.types.IntegerType * key = ['key] * functions = [COUNT(DISTINCT 'cat1), * COUNT(DISTINCT 'cat2), - * sum('value) with FILTER('id > 1)] + * sum('value) FILTER (WHERE 'id > 1)] * output = ['key, 'cat1_cnt, 'cat2_cnt, 'total]) * LocalTableScan [...] * }}} @@ -108,7 +108,7 @@ import org.apache.spark.sql.types.IntegerType * output = ['key, 'cat1_cnt, 'cat2_cnt, 'total]) * Aggregate( * key = ['key, 'cat1, 'cat2, 'gid] - * functions = [sum('value) with FILTER('id > 1)] + * functions = [sum('value) FILTER (WHERE 'id > 1)] * output = ['key, 'cat1, 'cat2, 'gid, 'total]) * Expand( * projections = [('key, null, null, 0, cast('value as bigint), 'id), @@ -118,6 +118,49 @@ import org.apache.spark.sql.types.IntegerType * LocalTableScan [...] * }}} * + * Third example: aggregate function with distinct and filter clauses (in sql): + * {{{ + * SELECT + * COUNT(DISTINCT cat1) FILTER (WHERE id > 1) as cat1_cnt, + * COUNT(DISTINCT cat2) FILTER (WHERE id > 2) as cat2_cnt, + * SUM(value) FILTER (WHERE id > 3) AS total + * FROM + * data + * GROUP BY + * key + * }}} + * + * This translates to the following (pseudo) logical plan: + * {{{ + * Aggregate( + * key = ['key] + * functions = [COUNT(DISTINCT 'cat1) FILTER (WHERE 'id > 1), + * COUNT(DISTINCT 'cat2) FILTER (WHERE 'id > 2), + * sum('value) FILTER (WHERE 'id > 3)] + * output = ['key, 'cat1_cnt, 'cat2_cnt, 'total]) + * LocalTableScan [...] + * }}} + * + * This rule rewrites this logical plan to the following (pseudo) logical plan: + * {{{ + * Aggregate( + * key = ['key] + * functions = [count(if (('gid = 1) and 'max_cond1) 'cat1 else null), + * count(if (('gid = 2) and 'max_cond2) 'cat2 else null), + * first(if (('gid = 0)) 'total else null) ignore nulls] + * output = ['key, 'cat1_cnt, 'cat2_cnt, 'total]) + * Aggregate( + * key = ['key, 'cat1, 'cat2, 'gid] + * functions = [max('cond1), max('cond2), sum('value) FILTER (WHERE 'id > 3)] + * output = ['key, 'cat1, 'cat2, 'gid, 'max_cond1, 'max_cond2, 'total]) + * Expand( + * projections = [('key, null, null, 0, null, null, cast('value as bigint), 'id), + * ('key, 'cat1, null, 1, 'id > 1, null, null, null), + * ('key, null, 'cat2, 2, null, 'id > 2, null, null)] + * output = ['key, 'cat1, 'cat2, 'gid, 'cond1, 'cond2, 'value, 'id]) + * LocalTableScan [...] + * }}} + * * The rule does the following things here: * 1. Expand the data. There are three aggregation groups in this query: * i. the non-distinct group; @@ -126,15 +169,24 @@ import org.apache.spark.sql.types.IntegerType * An expand operator is inserted to expand the child data for each group. The expand will null * out all unused columns for the given group; this must be done in order to ensure correctness * later on. Groups can by identified by a group id (gid) column added by the expand operator. + * If distinct group exists filter clause, the expand will calculate the filter and output it's + * result (e.g. cond1) which will be used to calculate the global conditions (e.g. max_cond1) + * equivalent to filter clauses. * 2. De-duplicate the distinct paths and aggregate the non-aggregate path. The group by clause of * this aggregate consists of the original group by clause, all the requested distinct columns * and the group id. Both de-duplication of distinct column and the aggregation of the * non-distinct group take advantage of the fact that we group by the group id (gid) and that we - * have nulled out all non-relevant columns the given group. + * have nulled out all non-relevant columns the given group. If distinct group exists filter + * clause, we will use max to aggregate the results (e.g. cond1) of the filter output in the + * previous step. These aggregate will output the global conditions (e.g. max_cond1) equivalent + * to filter clauses. * 3. Aggregating the distinct groups and combining this with the results of the non-distinct - * aggregation. In this step we use the group id to filter the inputs for the aggregate - * functions. The result of the non-distinct group are 'aggregated' by using the first operator, - * it might be more elegant to use the native UDAF merge mechanism for this in the future. + * aggregation. In this step we use the group id and the global condition to filter the inputs + * for the aggregate functions. If the global condition (e.g. max_cond1) is true, it means at + * least one row of a distinct value satisfies the filter. This distinct value should be included + * in the aggregate function. The result of the non-distinct group are 'aggregated' by using + * the first operator, it might be more elegant to use the native UDAF merge mechanism for this + * in the future. * * This rule duplicates the input data by two or more times (# distinct groups + an optional * non-distinct group). This will put quite a bit of memory pressure of the used aggregate and @@ -144,28 +196,24 @@ import org.apache.spark.sql.types.IntegerType */ object RewriteDistinctAggregates extends Rule[LogicalPlan] { - private def mayNeedtoRewrite(exprs: Seq[Expression]): Boolean = { - val distinctAggs = exprs.flatMap { _.collect { - case ae: AggregateExpression if ae.isDistinct => ae - }} - // We need at least two distinct aggregates for this rule because aggregation - // strategy can handle a single distinct group. + private def mayNeedtoRewrite(a: Aggregate): Boolean = { + val aggExpressions = collectAggregateExprs(a) + val distinctAggs = aggExpressions.filter(_.isDistinct) + // We need at least two distinct aggregates or the single distinct aggregate group exists filter + // clause for this rule because aggregation strategy can handle a single distinct aggregate + // group without filter clause. // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a). - distinctAggs.size > 1 + distinctAggs.size > 1 || distinctAggs.exists(_.filter.isDefined) } def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { - case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) => rewrite(a) + case a: Aggregate if mayNeedtoRewrite(a) => rewrite(a) } def rewrite(a: Aggregate): Aggregate = { - // Collect all aggregate expressions. - val aggExpressions = a.aggregateExpressions.flatMap { e => - e.collect { - case ae: AggregateExpression => ae - } - } + val aggExpressions = collectAggregateExprs(a) + val distinctAggs = aggExpressions.filter(_.isDistinct) // Extract distinct aggregate expressions. val distinctAggGroups = aggExpressions.filter(_.isDistinct).groupBy { e => @@ -184,8 +232,8 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] { } } - // Aggregation strategy can handle queries with a single distinct group. - if (distinctAggGroups.size > 1) { + // Aggregation strategy can handle queries with a single distinct group without filter clause. + if (distinctAggGroups.size > 1 || distinctAggs.exists(_.filter.isDefined)) { // Create the attributes for the grouping id and the group by clause. val gid = AttributeReference("gid", IntegerType, nullable = false)() val groupByMap = a.groupingExpressions.collect { @@ -195,7 +243,13 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] { val groupByAttrs = groupByMap.map(_._2) // Functions used to modify aggregate functions and their inputs. - def evalWithinGroup(id: Literal, e: Expression) = If(EqualTo(gid, id), e, nullify(e)) + def evalWithinGroup(id: Literal, e: Expression, condition: Option[Expression]) = + if (condition.isDefined) { + If(And(EqualTo(gid, id), condition.get), e, nullify(e)) + } else { + If(EqualTo(gid, id), e, nullify(e)) + } + def patchAggregateFunctionChildren( af: AggregateFunction)( attrs: Expression => Option[Expression]): AggregateFunction = { @@ -207,13 +261,28 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] { val distinctAggChildren = distinctAggGroups.keySet.flatten.toSeq.distinct val distinctAggChildAttrMap = distinctAggChildren.map(expressionAttributePair) val distinctAggChildAttrs = distinctAggChildAttrMap.map(_._2) + // Setup all the filters in distinct aggregate. + val (distinctAggFilters, distinctAggFilterAttrs, maxConds) = distinctAggs.collect { + case AggregateExpression(_, _, _, filter, _) if filter.isDefined => + val (e, attr) = expressionAttributePair(filter.get) + val aggregateExp = Max(attr).toAggregateExpression() + (e, attr, Alias(aggregateExp, attr.name)()) + }.unzip3 // Setup expand & aggregate operators for distinct aggregate expressions. val distinctAggChildAttrLookup = distinctAggChildAttrMap.toMap + val distinctAggFilterAttrLookup = distinctAggFilters.zip(maxConds.map(_.toAttribute)).toMap val distinctAggOperatorMap = distinctAggGroups.toSeq.zipWithIndex.map { case ((group, expressions), i) => val id = Literal(i + 1) + // Expand projection for filter + val filters = expressions.filter(_.filter.isDefined).map(_.filter.get) + val filterProjection = distinctAggFilters.map { + case e if filters.contains(e) => e + case e => nullify(e) + } + // Expand projection val projection = distinctAggChildren.map { case e if group.contains(e) => e @@ -224,12 +293,17 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] { val operators = expressions.map { e => val af = e.aggregateFunction val naf = patchAggregateFunctionChildren(af) { x => - distinctAggChildAttrLookup.get(x).map(evalWithinGroup(id, _)) + val condition = if (e.filter.isDefined) { + e.filter.map(distinctAggFilterAttrLookup.get(_)).get + } else { + None + } + distinctAggChildAttrLookup.get(x).map(evalWithinGroup(id, _, condition)) } - (e, e.copy(aggregateFunction = naf, isDistinct = false)) + (e, e.copy(aggregateFunction = naf, isDistinct = false, filter = None)) } - (projection, operators) + (projection ++ filterProjection, operators) } // Setup expand for the 'regular' aggregate expressions. @@ -257,7 +331,7 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] { // Select the result of the first aggregate in the last aggregate. val result = AggregateExpression( - aggregate.First(evalWithinGroup(regularGroupId, operator.toAttribute), true), + aggregate.First(evalWithinGroup(regularGroupId, operator.toAttribute, None), true), mode = Complete, isDistinct = false) @@ -280,6 +354,7 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] { Seq(a.groupingExpressions ++ distinctAggChildren.map(nullify) ++ Seq(regularGroupId) ++ + distinctAggFilters.map(nullify) ++ regularAggChildren) } else { Seq.empty[Seq[Expression]] @@ -297,7 +372,8 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] { // Construct the expand operator. val expand = Expand( regularAggProjection ++ distinctAggProjections, - groupByAttrs ++ distinctAggChildAttrs ++ Seq(gid) ++ regularAggChildAttrMap.map(_._2), + groupByAttrs ++ distinctAggChildAttrs ++ Seq(gid) ++ distinctAggFilterAttrs ++ + regularAggChildAttrMap.map(_._2), a.child) // Construct the first aggregate operator. This de-duplicates all the children of @@ -305,7 +381,7 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] { val firstAggregateGroupBy = groupByAttrs ++ distinctAggChildAttrs :+ gid val firstAggregate = Aggregate( firstAggregateGroupBy, - firstAggregateGroupBy ++ regularAggOperatorMap.map(_._2), + firstAggregateGroupBy ++ maxConds ++ regularAggOperatorMap.map(_._2), expand) // Construct the second aggregate @@ -331,6 +407,13 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] { } } + private def collectAggregateExprs(a: Aggregate): Seq[AggregateExpression] = { + // Collect all aggregate expressions. + a.aggregateExpressions.flatMap { _.collect { + case ae: AggregateExpression => ae + }} + } + private def nullify(e: Expression) = Literal.create(null, e.dataType) private def expressionAttributePair(e: Expression) = diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 6b407e65a9a72..f54e0192b6df8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2686,9 +2686,19 @@ object SQLConf { "optimized from O(M*N) calculation into O(M) calculation " + "using Hash lookup instead of Looping lookup." + "Only support for singleColumn NAAJ for now.") + .version("3.1.0") .booleanConf .createWithDefault(true) + val LEGACY_COMPLEX_TYPES_TO_STRING = + buildConf("spark.sql.legacy.castComplexTypesToString.enabled") + .internal() + .doc("When true, maps and structs are wrapped by [] in casting to strings. " + + "Otherwise, if this is false, which is the default, maps and structs are wrapped by {}.") + .version("3.1.0") + .booleanConf + .createWithDefault(false) + /** * Holds information about keys that have been deprecated. * diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala index 10f54f856a193..ca1074fcf6fc0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala @@ -127,6 +127,16 @@ object StaticSQLConf { .toSequence .createOptional + val SPARK_CACHE_SERIALIZER = buildStaticConf("spark.sql.cache.serializer") + .doc("The name of a class that implements " + + "org.apache.spark.sql.columnar.CachedBatchSerializer. It will be used to " + + "translate SQL data into a format that can more efficiently be cached. The underlying " + + "API is subject to change so use with caution. Multiple classes cannot be specified. " + + "The class must have a no-arg constructor.") + .version("3.1.0") + .stringConf + .createWithDefault("org.apache.spark.sql.execution.columnar.DefaultCachedBatchSerializer") + val QUERY_EXECUTION_LISTENERS = buildStaticConf("spark.sql.queryExecutionListeners") .doc("List of class names implementing QueryExecutionListener that will be automatically " + "added to newly created sessions. The classes should have either a no-arg constructor, " + diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 166ffec44a60d..a99f7e2be6e7e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -207,11 +207,6 @@ class AnalysisErrorSuite extends AnalysisTest { "FILTER (WHERE c > 1)"), "FILTER predicate specified, but aggregate is not an aggregate function" :: Nil) - errorTest( - "DISTINCT aggregate function with filter predicate", - CatalystSqlParser.parsePlan("SELECT count(DISTINCT a) FILTER (WHERE c > 1) FROM TaBlE2"), - "DISTINCT and FILTER cannot be used in aggregate functions at the same time" :: Nil) - errorTest( "non-deterministic filter predicate in aggregate functions", CatalystSqlParser.parsePlan("SELECT count(a) FILTER (WHERE rand(int(c)) > 1) FROM TaBlE2"), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala index 4ab288a34cb08..b8b93d929d39d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala @@ -712,47 +712,59 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { } test("SPARK-22973 Cast map to string") { - val ret1 = cast(Literal.create(Map(1 -> "a", 2 -> "b", 3 -> "c")), StringType) - checkEvaluation(ret1, "[1 -> a, 2 -> b, 3 -> c]") - val ret2 = cast( - Literal.create(Map("1" -> "a".getBytes, "2" -> null, "3" -> "c".getBytes)), - StringType) - checkEvaluation(ret2, "[1 -> a, 2 ->, 3 -> c]") - val ret3 = cast( - Literal.create(Map( - 1 -> Date.valueOf("2014-12-03"), - 2 -> Date.valueOf("2014-12-04"), - 3 -> Date.valueOf("2014-12-05"))), - StringType) - checkEvaluation(ret3, "[1 -> 2014-12-03, 2 -> 2014-12-04, 3 -> 2014-12-05]") - val ret4 = cast( - Literal.create(Map( - 1 -> Timestamp.valueOf("2014-12-03 13:01:00"), - 2 -> Timestamp.valueOf("2014-12-04 15:05:00"))), - StringType) - checkEvaluation(ret4, "[1 -> 2014-12-03 13:01:00, 2 -> 2014-12-04 15:05:00]") - val ret5 = cast( - Literal.create(Map( - 1 -> Array(1, 2, 3), - 2 -> Array(4, 5, 6))), - StringType) - checkEvaluation(ret5, "[1 -> [1, 2, 3], 2 -> [4, 5, 6]]") + Seq( + "false" -> ("{", "}"), + "true" -> ("[", "]")).foreach { case (legacyBrackets, (lb, rb)) => + withSQLConf(SQLConf.LEGACY_COMPLEX_TYPES_TO_STRING.key -> legacyBrackets) { + val ret1 = cast(Literal.create(Map(1 -> "a", 2 -> "b", 3 -> "c")), StringType) + checkEvaluation(ret1, s"${lb}1 -> a, 2 -> b, 3 -> c$rb") + val ret2 = cast( + Literal.create(Map("1" -> "a".getBytes, "2" -> null, "3" -> "c".getBytes)), + StringType) + checkEvaluation(ret2, s"${lb}1 -> a, 2 ->, 3 -> c$rb") + val ret3 = cast( + Literal.create(Map( + 1 -> Date.valueOf("2014-12-03"), + 2 -> Date.valueOf("2014-12-04"), + 3 -> Date.valueOf("2014-12-05"))), + StringType) + checkEvaluation(ret3, s"${lb}1 -> 2014-12-03, 2 -> 2014-12-04, 3 -> 2014-12-05$rb") + val ret4 = cast( + Literal.create(Map( + 1 -> Timestamp.valueOf("2014-12-03 13:01:00"), + 2 -> Timestamp.valueOf("2014-12-04 15:05:00"))), + StringType) + checkEvaluation(ret4, s"${lb}1 -> 2014-12-03 13:01:00, 2 -> 2014-12-04 15:05:00$rb") + val ret5 = cast( + Literal.create(Map( + 1 -> Array(1, 2, 3), + 2 -> Array(4, 5, 6))), + StringType) + checkEvaluation(ret5, s"${lb}1 -> [1, 2, 3], 2 -> [4, 5, 6]$rb") + } + } } test("SPARK-22981 Cast struct to string") { - val ret1 = cast(Literal.create((1, "a", 0.1)), StringType) - checkEvaluation(ret1, "[1, a, 0.1]") - val ret2 = cast(Literal.create(Tuple3[Int, String, String](1, null, "a")), StringType) - checkEvaluation(ret2, "[1,, a]") - val ret3 = cast(Literal.create( - (Date.valueOf("2014-12-03"), Timestamp.valueOf("2014-12-03 15:05:00"))), StringType) - checkEvaluation(ret3, "[2014-12-03, 2014-12-03 15:05:00]") - val ret4 = cast(Literal.create(((1, "a"), 5, 0.1)), StringType) - checkEvaluation(ret4, "[[1, a], 5, 0.1]") - val ret5 = cast(Literal.create((Seq(1, 2, 3), "a", 0.1)), StringType) - checkEvaluation(ret5, "[[1, 2, 3], a, 0.1]") - val ret6 = cast(Literal.create((1, Map(1 -> "a", 2 -> "b", 3 -> "c"))), StringType) - checkEvaluation(ret6, "[1, [1 -> a, 2 -> b, 3 -> c]]") + Seq( + "false" -> ("{", "}"), + "true" -> ("[", "]")).foreach { case (legacyBrackets, (lb, rb)) => + withSQLConf(SQLConf.LEGACY_COMPLEX_TYPES_TO_STRING.key -> legacyBrackets) { + val ret1 = cast(Literal.create((1, "a", 0.1)), StringType) + checkEvaluation(ret1, s"${lb}1, a, 0.1$rb") + val ret2 = cast(Literal.create(Tuple3[Int, String, String](1, null, "a")), StringType) + checkEvaluation(ret2, s"${lb}1,, a$rb") + val ret3 = cast(Literal.create( + (Date.valueOf("2014-12-03"), Timestamp.valueOf("2014-12-03 15:05:00"))), StringType) + checkEvaluation(ret3, s"${lb}2014-12-03, 2014-12-03 15:05:00$rb") + val ret4 = cast(Literal.create(((1, "a"), 5, 0.1)), StringType) + checkEvaluation(ret4, s"$lb${lb}1, a$rb, 5, 0.1$rb") + val ret5 = cast(Literal.create((Seq(1, 2, 3), "a", 0.1)), StringType) + checkEvaluation(ret5, s"$lb[1, 2, 3], a, 0.1$rb") + val ret6 = cast(Literal.create((1, Map(1 -> "a", 2 -> "b", 3 -> "c"))), StringType) + checkEvaluation(ret6, s"${lb}1, ${lb}1 -> a, 2 -> b, 3 -> c$rb$rb") + } + } } test("up-cast") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala index cab81f85fda06..205dc10efc8a8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala @@ -322,6 +322,56 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { RegExpExtract(Literal("\"quote"), Literal("\"quote"), Literal(1)) :: Nil) } + test("RegexExtractAll") { + val row1 = create_row("100-200,300-400,500-600", "(\\d+)-(\\d+)", 0) + val row2 = create_row("100-200,300-400,500-600", "(\\d+)-(\\d+)", 1) + val row3 = create_row("100-200,300-400,500-600", "(\\d+)-(\\d+)", 2) + val row4 = create_row("100-200,300-400,500-600", "(\\d+).*", 1) + val row5 = create_row("100-200,300-400,500-600", "([a-z])", 1) + val row6 = create_row(null, "([a-z])", 1) + val row7 = create_row("100-200,300-400,500-600", null, 1) + val row8 = create_row("100-200,300-400,500-600", "([a-z])", null) + + val s = 's.string.at(0) + val p = 'p.string.at(1) + val r = 'r.int.at(2) + + val expr = RegExpExtractAll(s, p, r) + checkEvaluation(expr, Seq("100-200", "300-400", "500-600"), row1) + checkEvaluation(expr, Seq("100", "300", "500"), row2) + checkEvaluation(expr, Seq("200", "400", "600"), row3) + checkEvaluation(expr, Seq("100"), row4) + checkEvaluation(expr, Seq(), row5) + checkEvaluation(expr, null, row6) + checkEvaluation(expr, null, row7) + checkEvaluation(expr, null, row8) + + val expr1 = new RegExpExtractAll(s, p) + checkEvaluation(expr1, Seq("100", "300", "500"), row2) + + val nonNullExpr = RegExpExtractAll(Literal("100-200,300-400,500-600"), + Literal("(\\d+)-(\\d+)"), Literal(1)) + checkEvaluation(nonNullExpr, Seq("100", "300", "500"), row2) + + // invalid group index + val row9 = create_row("100-200,300-400,500-600", "(\\d+)-(\\d+)", 3) + val row10 = create_row("100-200,300-400,500-600", "(\\d+).*", 2) + val row11 = create_row("100-200,300-400,500-600", "\\d+", 1) + val row12 = create_row("100-200,300-400,500-600", "(\\d+)-(\\d+)", -1) + val row13 = create_row("100-200,300-400,500-600", "\\d+", -1) + + checkExceptionInExpression[IllegalArgumentException]( + expr, row9, "Regex group count is 2, but the specified group index is 3") + checkExceptionInExpression[IllegalArgumentException]( + expr, row10, "Regex group count is 1, but the specified group index is 2") + checkExceptionInExpression[IllegalArgumentException]( + expr, row11, "Regex group count is 0, but the specified group index is 1") + checkExceptionInExpression[IllegalArgumentException]( + expr, row12, "The specified group index cannot be less than zero") + checkExceptionInExpression[IllegalArgumentException]( + expr, row13, "The specified group index cannot be less than zero") + } + test("SPLIT") { val s1 = 'a.string.at(0) val s2 = 'b.string.at(1) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 306c3235b0bc0..e5d53f5fd4c65 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -29,7 +29,7 @@ import org.apache.spark.{SPARK_VERSION, SparkConf, SparkContext, TaskContext} import org.apache.spark.annotation.{DeveloperApi, Experimental, Stable, Unstable} import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.ALLOW_SPARK_CONTEXT_IN_EXECUTORS +import org.apache.spark.internal.config.EXECUTOR_ALLOW_SPARK_CONTEXT import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd} import org.apache.spark.sql.catalog.Catalog @@ -904,7 +904,7 @@ object SparkSession extends Logging { val sparkConf = new SparkConf() options.foreach { case (k, v) => sparkConf.set(k, v) } - if (!sparkConf.get(ALLOW_SPARK_CONTEXT_IN_EXECUTORS)) { + if (!sparkConf.get(EXECUTOR_ALLOW_SPARK_CONTEXT)) { assertOnDriver() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala new file mode 100644 index 0000000000000..1113e63cab332 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala @@ -0,0 +1,343 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.columnar + +import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeReference, BindReferences, EqualNullSafe, EqualTo, Expression, GreaterThan, GreaterThanOrEqual, In, IsNotNull, IsNull, Length, LessThan, LessThanOrEqual, Literal, Or, Predicate, StartsWith} +import org.apache.spark.sql.execution.columnar.{ColumnStatisticsSchema, PartitionStatistics} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{AtomicType, BinaryType, StructType} +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.storage.StorageLevel + +/** + * Basic interface that all cached batches of data must support. This is primarily to allow + * for metrics to be handled outside of the encoding and decoding steps in a standard way. + */ +@DeveloperApi +@Since("3.1.0") +trait CachedBatch { + def numRows: Int + def sizeInBytes: Long +} + +/** + * Provides APIs that handle transformations of SQL data associated with the cache/persist APIs. + */ +@DeveloperApi +@Since("3.1.0") +trait CachedBatchSerializer extends Serializable { + /** + * Can `convertColumnarBatchToCachedBatch()` be called instead of + * `convertInternalRowToCachedBatch()` for this given schema? True if it can and false if it + * cannot. Columnar input is only supported if the plan could produce columnar output. Currently + * this is mostly supported by input formats like parquet and orc, but more operations are likely + * to be supported soon. + * @param schema the schema of the data being stored. + * @return True if columnar input can be supported, else false. + */ + def supportsColumnarInput(schema: Seq[Attribute]): Boolean + + /** + * Convert an `RDD[InternalRow]` into an `RDD[CachedBatch]` in preparation for caching the data. + * @param input the input `RDD` to be converted. + * @param schema the schema of the data being stored. + * @param storageLevel where the data will be stored. + * @param conf the config for the query. + * @return The data converted into a format more suitable for caching. + */ + def convertInternalRowToCachedBatch( + input: RDD[InternalRow], + schema: Seq[Attribute], + storageLevel: StorageLevel, + conf: SQLConf): RDD[CachedBatch] + + /** + * Convert an `RDD[ColumnarBatch]` into an `RDD[CachedBatch]` in preparation for caching the data. + * This will only be called if `supportsColumnarInput()` returned true for the given schema and + * the plan up to this point would could produce columnar output without modifying it. + * @param input the input `RDD` to be converted. + * @param schema the schema of the data being stored. + * @param storageLevel where the data will be stored. + * @param conf the config for the query. + * @return The data converted into a format more suitable for caching. + */ + def convertColumnarBatchToCachedBatch( + input: RDD[ColumnarBatch], + schema: Seq[Attribute], + storageLevel: StorageLevel, + conf: SQLConf): RDD[CachedBatch] + + /** + * Builds a function that can be used to filter batches prior to being decompressed. + * In most cases extending [[SimpleMetricsCachedBatchSerializer]] will provide the filter logic + * necessary. You will need to provide metrics for this to work. [[SimpleMetricsCachedBatch]] + * provides the APIs to hold those metrics and explains the metrics used, really just min and max. + * Note that this is intended to skip batches that are not needed, and the actual filtering of + * individual rows is handled later. + * @param predicates the set of expressions to use for filtering. + * @param cachedAttributes the schema/attributes of the data that is cached. This can be helpful + * if you don't store it with the data. + * @return a function that takes the partition id and the iterator of batches in the partition. + * It returns an iterator of batches that should be decompressed. + */ + def buildFilter( + predicates: Seq[Expression], + cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] + + /** + * Can `convertCachedBatchToColumnarBatch()` be called instead of + * `convertCachedBatchToInternalRow()` for this given schema? True if it can and false if it + * cannot. Columnar output is typically preferred because it is more efficient. Note that + * `convertCachedBatchToInternalRow()` must always be supported as there are other checks that + * can force row based output. + * @param schema the schema of the data being checked. + * @return true if columnar output should be used for this schema, else false. + */ + def supportsColumnarOutput(schema: StructType): Boolean + + /** + * The exact java types of the columns that are output in columnar processing mode. This + * is a performance optimization for code generation and is optional. + * @param attributes the attributes to be output. + * @param conf the config for the query that will read the data. + */ + def vectorTypes(attributes: Seq[Attribute], conf: SQLConf): Option[Seq[String]] = None + + /** + * Convert the cached data into a ColumnarBatch. This currently is only used if + * `supportsColumnarOutput()` returns true for the associated schema, but there are other checks + * that can force row based output. One of the main advantages of doing columnar output over row + * based output is that the code generation is more standard and can be combined with code + * generation for downstream operations. + * @param input the cached batches that should be converted. + * @param cacheAttributes the attributes of the data in the batch. + * @param selectedAttributes the fields that should be loaded from the data and the order they + * should appear in the output batch. + * @param conf the configuration for the job. + * @return an RDD of the input cached batches transformed into the ColumnarBatch format. + */ + def convertCachedBatchToColumnarBatch( + input: RDD[CachedBatch], + cacheAttributes: Seq[Attribute], + selectedAttributes: Seq[Attribute], + conf: SQLConf): RDD[ColumnarBatch] + + /** + * Convert the cached batch into `InternalRow`s. If you want this to be performant, code + * generation is advised. + * @param input the cached batches that should be converted. + * @param cacheAttributes the attributes of the data in the batch. + * @param selectedAttributes the field that should be loaded from the data and the order they + * should appear in the output rows. + * @param conf the configuration for the job. + * @return RDD of the rows that were stored in the cached batches. + */ + def convertCachedBatchToInternalRow( + input: RDD[CachedBatch], + cacheAttributes: Seq[Attribute], + selectedAttributes: Seq[Attribute], + conf: SQLConf): RDD[InternalRow] +} + +/** + * A [[CachedBatch]] that stores some simple metrics that can be used for filtering of batches with + * the [[SimpleMetricsCachedBatchSerializer]]. + * The metrics are returned by the stats value. For each column in the batch 5 columns of metadata + * are needed in the row. + */ +@DeveloperApi +@Since("3.1.0") +trait SimpleMetricsCachedBatch extends CachedBatch { + /** + * Holds stats for each cached column. The optional `upperBound` and `lowerBound` should be + * of the same type as the original column. If they are null, then it is assumed that they + * are not provided, and will not be used for filtering. + *
    + *
  • `upperBound` (optional)
  • + *
  • `lowerBound` (Optional)
  • + *
  • `nullCount`: `Int`
  • + *
  • `rowCount`: `Int`
  • + *
  • `sizeInBytes`: `Long`
  • + *
+ * These are repeated for each column in the original cached data. + */ + val stats: InternalRow + override def sizeInBytes: Long = + Range.apply(4, stats.numFields, 5).map(stats.getLong).sum +} + +// Currently, uses statistics for all atomic types that are not `BinaryType`. +private object ExtractableLiteral { + def unapply(expr: Expression): Option[Literal] = expr match { + case lit: Literal => lit.dataType match { + case BinaryType => None + case _: AtomicType => Some(lit) + case _ => None + } + case _ => None + } +} + +/** + * Provides basic filtering for [[CachedBatchSerializer]] implementations. + * The requirement to extend this is that all of the batches produced by your serializer are + * instances of [[SimpleMetricsCachedBatch]]. + * This does not calculate the metrics needed to be stored in the batches. That is up to each + * implementation. The metrics required are really just min and max values and those are optional + * especially for complex types. Because those metrics are simple and it is likely that compression + * will also be done on the data we thought it best to let each implementation decide on the most + * efficient way to calculate the metrics, possibly combining them with compression passes that + * might also be done across the data. + */ +@DeveloperApi +@Since("3.1.0") +abstract class SimpleMetricsCachedBatchSerializer extends CachedBatchSerializer with Logging { + override def buildFilter( + predicates: Seq[Expression], + cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] = { + val stats = new PartitionStatistics(cachedAttributes) + val statsSchema = stats.schema + + def statsFor(a: Attribute): ColumnStatisticsSchema = { + stats.forAttribute(a) + } + + // Returned filter predicate should return false iff it is impossible for the input expression + // to evaluate to `true` based on statistics collected about this partition batch. + @transient lazy val buildFilter: PartialFunction[Expression, Expression] = { + case And(lhs: Expression, rhs: Expression) + if buildFilter.isDefinedAt(lhs) || buildFilter.isDefinedAt(rhs) => + (buildFilter.lift(lhs) ++ buildFilter.lift(rhs)).reduce(_ && _) + + case Or(lhs: Expression, rhs: Expression) + if buildFilter.isDefinedAt(lhs) && buildFilter.isDefinedAt(rhs) => + buildFilter(lhs) || buildFilter(rhs) + + case EqualTo(a: AttributeReference, ExtractableLiteral(l)) => + statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound + case EqualTo(ExtractableLiteral(l), a: AttributeReference) => + statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound + + case EqualNullSafe(a: AttributeReference, ExtractableLiteral(l)) => + statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound + case EqualNullSafe(ExtractableLiteral(l), a: AttributeReference) => + statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound + + case LessThan(a: AttributeReference, ExtractableLiteral(l)) => statsFor(a).lowerBound < l + case LessThan(ExtractableLiteral(l), a: AttributeReference) => l < statsFor(a).upperBound + + case LessThanOrEqual(a: AttributeReference, ExtractableLiteral(l)) => + statsFor(a).lowerBound <= l + case LessThanOrEqual(ExtractableLiteral(l), a: AttributeReference) => + l <= statsFor(a).upperBound + + case GreaterThan(a: AttributeReference, ExtractableLiteral(l)) => l < statsFor(a).upperBound + case GreaterThan(ExtractableLiteral(l), a: AttributeReference) => statsFor(a).lowerBound < l + + case GreaterThanOrEqual(a: AttributeReference, ExtractableLiteral(l)) => + l <= statsFor(a).upperBound + case GreaterThanOrEqual(ExtractableLiteral(l), a: AttributeReference) => + statsFor(a).lowerBound <= l + + case IsNull(a: Attribute) => statsFor(a).nullCount > 0 + case IsNotNull(a: Attribute) => statsFor(a).count - statsFor(a).nullCount > 0 + + case In(a: AttributeReference, list: Seq[Expression]) + if list.forall(ExtractableLiteral.unapply(_).isDefined) && list.nonEmpty => + list.map(l => statsFor(a).lowerBound <= l.asInstanceOf[Literal] && + l.asInstanceOf[Literal] <= statsFor(a).upperBound).reduce(_ || _) + // This is an example to explain how it works, imagine that the id column stored as follows: + // __________________________________________ + // | Partition ID | lowerBound | upperBound | + // |--------------|------------|------------| + // | p1 | '1' | '9' | + // | p2 | '10' | '19' | + // | p3 | '20' | '29' | + // | p4 | '30' | '39' | + // | p5 | '40' | '49' | + // |______________|____________|____________| + // + // A filter: df.filter($"id".startsWith("2")). + // In this case it substr lowerBound and upperBound: + // ________________________________________________________________________________________ + // | Partition ID | lowerBound.substr(0, Length("2")) | upperBound.substr(0, Length("2")) | + // |--------------|-----------------------------------|-----------------------------------| + // | p1 | '1' | '9' | + // | p2 | '1' | '1' | + // | p3 | '2' | '2' | + // | p4 | '3' | '3' | + // | p5 | '4' | '4' | + // |______________|___________________________________|___________________________________| + // + // We can see that we only need to read p1 and p3. + case StartsWith(a: AttributeReference, ExtractableLiteral(l)) => + statsFor(a).lowerBound.substr(0, Length(l)) <= l && + l <= statsFor(a).upperBound.substr(0, Length(l)) + } + + // When we bind the filters we need to do it against the stats schema + val partitionFilters: Seq[Expression] = { + predicates.flatMap { p => + val filter = buildFilter.lift(p) + val boundFilter = + filter.map( + BindReferences.bindReference( + _, + statsSchema, + allowFailures = true)) + + boundFilter.foreach(_ => + filter.foreach(f => logInfo(s"Predicate $p generates partition filter: $f"))) + + // If the filter can't be resolved then we are missing required statistics. + boundFilter.filter(_.resolved) + } + } + + def ret(index: Int, cachedBatchIterator: Iterator[CachedBatch]): Iterator[CachedBatch] = { + val partitionFilter = Predicate.create( + partitionFilters.reduceOption(And).getOrElse(Literal(true)), + cachedAttributes) + + partitionFilter.initialize(index) + val schemaIndex = cachedAttributes.zipWithIndex + + cachedBatchIterator.filter { cb => + val cachedBatch = cb.asInstanceOf[SimpleMetricsCachedBatch] + if (!partitionFilter.eval(cachedBatch.stats)) { + logDebug { + val statsString = schemaIndex.map { case (a, i) => + val value = cachedBatch.stats.get(i, a.dataType) + s"${a.name}: $value" + }.mkString(", ") + s"Skipping partition based on stats $statsString" + } + false + } else { + true + } + } + } + ret + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala index 7d86c48015406..7201026b11b6b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala @@ -27,11 +27,10 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, SubqueryExpression} import org.apache.spark.sql.catalyst.optimizer.EliminateResolvedHint import org.apache.spark.sql.catalyst.plans.logical.{IgnoreCachedData, LogicalPlan, ResolvedHint} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper -import org.apache.spark.sql.execution.columnar.InMemoryRelation +import org.apache.spark.sql.execution.columnar.{DefaultCachedBatchSerializer, InMemoryRelation} import org.apache.spark.sql.execution.command.CommandUtils import org.apache.spark.sql.execution.datasources.{FileIndex, HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, FileTable} -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.storage.StorageLevel import org.apache.spark.storage.StorageLevel.MEMORY_AND_DISK @@ -85,11 +84,9 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper { val inMemoryRelation = sessionWithAqeOff.withActive { val qe = sessionWithAqeOff.sessionState.executePlan(planToCache) InMemoryRelation( - sessionWithAqeOff.sessionState.conf.useCompression, - sessionWithAqeOff.sessionState.conf.columnBatchSize, storageLevel, - qe.executedPlan, - tableName, - optimizedPlan = qe.optimizedPlan) + storageLevel, + qe, + tableName) } this.synchronized { @@ -195,9 +192,7 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper { val sessionWithAqeOff = getOrCloneSessionWithAqeOff(spark) val newCache = sessionWithAqeOff.withActive { val qe = sessionWithAqeOff.sessionState.executePlan(cd.plan) - InMemoryRelation( - cacheBuilder = cd.cachedRepresentation.cacheBuilder.copy(cachedPlan = qe.executedPlan), - optimizedPlan = qe.optimizedPlan) + InMemoryRelation(cd.cachedRepresentation.cacheBuilder, qe) } val recomputedPlan = cd.copy(cachedRepresentation = newCache) this.synchronized { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala index e01cd8598db0c..13ea609f7bfa6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala @@ -49,6 +49,13 @@ class ColumnarRule { def postColumnarTransitions: Rule[SparkPlan] = plan => plan } +/** + * A trait that is used as a tag to indicate a transition from columns to rows. This allows plugins + * to replace the current [[ColumnarToRowExec]] with an optimized version and still have operations + * that walk a spark plan looking for this type of transition properly match it. + */ +trait ColumnarToRowTransition extends UnaryExecNode + /** * Provides a common executor to translate an [[RDD]] of [[ColumnarBatch]] into an [[RDD]] of * [[InternalRow]]. This is inserted whenever such a transition is determined to be needed. @@ -57,7 +64,7 @@ class ColumnarRule { * [[org.apache.spark.sql.execution.python.ArrowEvalPythonExec]] and * [[MapPartitionsInRWithArrowExec]]. Eventually this should replace those implementations. */ -case class ColumnarToRowExec(child: SparkPlan) extends UnaryExecNode with CodegenSupport { +case class ColumnarToRowExec(child: SparkPlan) extends ColumnarToRowTransition with CodegenSupport { assert(child.supportsColumnar) override def output: Seq[Attribute] = child.output @@ -479,7 +486,9 @@ case class RowToColumnarExec(child: SparkPlan) extends UnaryExecNode { * Apply any user defined [[ColumnarRule]]s and find the correct place to insert transitions * to/from columnar formatted data. */ -case class ApplyColumnarRulesAndInsertTransitions(conf: SQLConf, columnarRules: Seq[ColumnarRule]) +case class ApplyColumnarRulesAndInsertTransitions( + conf: SQLConf, + columnarRules: Seq[ColumnarRule]) extends Rule[SparkPlan] { /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 78808ff21394c..bef9f4b46c628 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -608,12 +608,20 @@ case class FileSourceScanExec( new FileScanRDD(fsRelation.sparkSession, readFile, partitions) } + // Filters unused DynamicPruningExpression expressions - one which has been replaced + // with DynamicPruningExpression(Literal.TrueLiteral) during Physical Planning + private def filterUnusedDynamicPruningExpressions( + predicates: Seq[Expression]): Seq[Expression] = { + predicates.filterNot(_ == DynamicPruningExpression(Literal.TrueLiteral)) + } + override def doCanonicalize(): FileSourceScanExec = { FileSourceScanExec( relation, output.map(QueryPlan.normalizeExpressions(_, output)), requiredSchema, - QueryPlan.normalizePredicates(partitionFilters, output), + QueryPlan.normalizePredicates( + filterUnusedDynamicPruningExpressions(partitionFilters), output), optionalBucketSet, optionalNumCoalescedBuckets, QueryPlan.normalizePredicates(dataFilters, output), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 012ae0a76043c..129312160b1b7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -58,6 +58,9 @@ class SparkSqlParser(conf: SQLConf) extends AbstractSqlParser(conf) { class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { import org.apache.spark.sql.catalyst.parser.ParserUtils._ + private val configKeyValueDef = """([a-zA-Z_\d\\.:]+)\s*=(.*)""".r + private val configKeyDef = """([a-zA-Z_\d\\.:]+)$""".r + /** * Create a [[SetCommand]] logical plan. * @@ -66,17 +69,28 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { * character in the raw string. */ override def visitSetConfiguration(ctx: SetConfigurationContext): LogicalPlan = withOrigin(ctx) { - // Construct the command. - val raw = remainder(ctx.SET.getSymbol) - val keyValueSeparatorIndex = raw.indexOf('=') - if (keyValueSeparatorIndex >= 0) { - val key = raw.substring(0, keyValueSeparatorIndex).trim - val value = raw.substring(keyValueSeparatorIndex + 1).trim - SetCommand(Some(key -> Option(value))) - } else if (raw.nonEmpty) { - SetCommand(Some(raw.trim -> None)) + remainder(ctx.SET.getSymbol).trim match { + case configKeyValueDef(key, value) => + SetCommand(Some(key -> Option(value.trim))) + case configKeyDef(key) => + SetCommand(Some(key -> None)) + case s if s == "-v" => + SetCommand(Some("-v" -> None)) + case s if s.isEmpty => + SetCommand(None) + case _ => throw new ParseException("Expected format is 'SET', 'SET key', or " + + "'SET key=value'. If you want to include special characters in key, " + + "please use quotes, e.g., SET `ke y`=value.", ctx) + } + } + + override def visitSetQuotedConfiguration(ctx: SetQuotedConfigurationContext) + : LogicalPlan = withOrigin(ctx) { + val keyStr = ctx.configKey().getText + if (ctx.EQ() != null) { + SetCommand(Some(keyStr -> Option(remainder(ctx.EQ().getSymbol).trim))) } else { - SetCommand(None) + SetCommand(Some(keyStr -> None)) } } @@ -90,7 +104,20 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { */ override def visitResetConfiguration( ctx: ResetConfigurationContext): LogicalPlan = withOrigin(ctx) { - ResetCommand(Option(remainder(ctx.RESET().getSymbol).trim).filter(_.nonEmpty)) + remainder(ctx.RESET.getSymbol).trim match { + case configKeyDef(key) => + ResetCommand(Some(key)) + case s if s.trim.isEmpty => + ResetCommand(None) + case _ => throw new ParseException("Expected format is 'RESET' or 'RESET key'. " + + "If you want to include special characters in key, " + + "please use quotes, e.g., RESET `ke y`.", ctx) + } + } + + override def visitResetQuotedConfiguration( + ctx: ResetQuotedConfigurationContext): LogicalPlan = withOrigin(ctx) { + ResetCommand(Some(ctx.configKey().getText)) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala index 20ecc57c49e75..45557bfbada6c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, Attri import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} -private[columnar] class ColumnStatisticsSchema(a: Attribute) extends Serializable { +class ColumnStatisticsSchema(a: Attribute) extends Serializable { val upperBound = AttributeReference(a.name + ".upperBound", a.dataType, nullable = true)() val lowerBound = AttributeReference(a.name + ".lowerBound", a.dataType, nullable = true)() val nullCount = AttributeReference(a.name + ".nullCount", IntegerType, nullable = false)() @@ -32,7 +32,7 @@ private[columnar] class ColumnStatisticsSchema(a: Attribute) extends Serializabl val schema = Seq(lowerBound, upperBound, nullCount, count, sizeInBytes) } -private[columnar] class PartitionStatistics(tableSchema: Seq[Attribute]) extends Serializable { +class PartitionStatistics(tableSchema: Seq[Attribute]) extends Serializable { val (forAttribute: AttributeMap[ColumnStatisticsSchema], schema: Seq[AttributeReference]) = { val allStats = tableSchema.map(a => a -> new ColumnStatisticsSchema(a)) (AttributeMap(allStats), allStats.flatMap(_._2.schema)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala index bd2d06665a910..eb0663830dd67 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala @@ -28,7 +28,7 @@ import org.apache.spark.unsafe.types.CalendarInterval * An Iterator to walk through the InternalRows from a CachedBatch */ abstract class ColumnarIterator extends Iterator[InternalRow] { - def initialize(input: Iterator[CachedBatch], columnTypes: Array[DataType], + def initialize(input: Iterator[DefaultCachedBatch], columnTypes: Array[DataType], columnIndexes: Array[Int]): Unit } @@ -203,7 +203,8 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera return false; } - ${classOf[CachedBatch].getName} batch = (${classOf[CachedBatch].getName}) input.next(); + ${classOf[DefaultCachedBatch].getName} batch = + (${classOf[DefaultCachedBatch].getName}) input.next(); currentRow = 0; numRowsInBatch = batch.numRows(); for (int i = 0; i < columnIndexes.length; i ++) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index cf9f3ddeb42a2..07411c0d3803c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -19,33 +19,189 @@ package org.apache.spark.sql.execution.columnar import org.apache.commons.lang3.StringUtils +import org.apache.spark.TaskContext import org.apache.spark.network.util.JavaUtils import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.QueryPlan -import org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan} import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics} import org.apache.spark.sql.catalyst.util.truncatedString -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.columnar.{CachedBatch, CachedBatchSerializer, SimpleMetricsCachedBatch, SimpleMetricsCachedBatchSerializer} +import org.apache.spark.sql.execution.{ColumnarToRowTransition, InputAdapter, QueryExecution, SparkPlan, WholeStageCodegenExec} +import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector} +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} +import org.apache.spark.sql.types.{BooleanType, ByteType, DoubleType, FloatType, IntegerType, LongType, ShortType, StructType, UserDefinedType} +import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.LongAccumulator - +import org.apache.spark.util.{LongAccumulator, Utils} /** - * CachedBatch is a cached batch of rows. + * The default implementation of CachedBatch. * * @param numRows The total number of rows in this batch * @param buffers The buffers for serialized columns * @param stats The stat of columns */ -private[columnar] -case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow) +case class DefaultCachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow) + extends SimpleMetricsCachedBatch + +/** + * The default implementation of CachedBatchSerializer. + */ +class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer { + override def supportsColumnarInput(schema: Seq[Attribute]): Boolean = false + + override def convertColumnarBatchToCachedBatch( + input: RDD[ColumnarBatch], + schema: Seq[Attribute], + storageLevel: StorageLevel, + conf: SQLConf): RDD[CachedBatch] = + throw new IllegalStateException("Columnar input is not supported") + + override def convertInternalRowToCachedBatch( + input: RDD[InternalRow], + schema: Seq[Attribute], + storageLevel: StorageLevel, + conf: SQLConf): RDD[CachedBatch] = { + val batchSize = conf.columnBatchSize + val useCompression = conf.useCompression + convertForCacheInternal(input, schema, batchSize, useCompression) + } + + def convertForCacheInternal( + input: RDD[InternalRow], + output: Seq[Attribute], + batchSize: Int, + useCompression: Boolean): RDD[CachedBatch] = { + input.mapPartitionsInternal { rowIterator => + new Iterator[DefaultCachedBatch] { + def next(): DefaultCachedBatch = { + val columnBuilders = output.map { attribute => + ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression) + }.toArray + + var rowCount = 0 + var totalSize = 0L + while (rowIterator.hasNext && rowCount < batchSize + && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) { + val row = rowIterator.next() + + // Added for SPARK-6082. This assertion can be useful for scenarios when something + // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM + // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat + // hard to decipher. + assert( + row.numFields == columnBuilders.length, + s"Row column number mismatch, expected ${output.size} columns, " + + s"but got ${row.numFields}." + + s"\nRow content: $row") + + var i = 0 + totalSize = 0 + while (i < row.numFields) { + columnBuilders(i).appendFrom(row, i) + totalSize += columnBuilders(i).columnStats.sizeInBytes + i += 1 + } + rowCount += 1 + } + + val stats = InternalRow.fromSeq( + columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq) + DefaultCachedBatch(rowCount, columnBuilders.map { builder => + JavaUtils.bufferToArray(builder.build()) + }, stats) + } + + def hasNext: Boolean = rowIterator.hasNext + } + } + } + + override def supportsColumnarOutput(schema: StructType): Boolean = schema.fields.forall(f => + f.dataType match { + // More types can be supported, but this is to match the original implementation that + // only supported primitive types "for ease of review" + case BooleanType | ByteType | ShortType | IntegerType | LongType | + FloatType | DoubleType => true + case _ => false + }) + + override def vectorTypes(attributes: Seq[Attribute], conf: SQLConf): Option[Seq[String]] = + Option(Seq.fill(attributes.length)( + if (!conf.offHeapColumnVectorEnabled) { + classOf[OnHeapColumnVector].getName + } else { + classOf[OffHeapColumnVector].getName + } + )) + + override def convertCachedBatchToColumnarBatch( + input: RDD[CachedBatch], + cacheAttributes: Seq[Attribute], + selectedAttributes: Seq[Attribute], + conf: SQLConf): RDD[ColumnarBatch] = { + val offHeapColumnVectorEnabled = conf.offHeapColumnVectorEnabled + val outputSchema = StructType.fromAttributes(selectedAttributes) + val columnIndices = + selectedAttributes.map(a => cacheAttributes.map(o => o.exprId).indexOf(a.exprId)).toArray + + def createAndDecompressColumn(cb: CachedBatch): ColumnarBatch = { + val cachedColumnarBatch = cb.asInstanceOf[DefaultCachedBatch] + val rowCount = cachedColumnarBatch.numRows + val taskContext = Option(TaskContext.get()) + val columnVectors = if (!offHeapColumnVectorEnabled || taskContext.isEmpty) { + OnHeapColumnVector.allocateColumns(rowCount, outputSchema) + } else { + OffHeapColumnVector.allocateColumns(rowCount, outputSchema) + } + val columnarBatch = new ColumnarBatch(columnVectors.asInstanceOf[Array[ColumnVector]]) + columnarBatch.setNumRows(rowCount) + + for (i <- selectedAttributes.indices) { + ColumnAccessor.decompress( + cachedColumnarBatch.buffers(columnIndices(i)), + columnarBatch.column(i).asInstanceOf[WritableColumnVector], + outputSchema.fields(i).dataType, rowCount) + } + taskContext.foreach(_.addTaskCompletionListener[Unit](_ => columnarBatch.close())) + columnarBatch + } + + input.map(createAndDecompressColumn) + } + + override def convertCachedBatchToInternalRow( + input: RDD[CachedBatch], + cacheAttributes: Seq[Attribute], + selectedAttributes: Seq[Attribute], + conf: SQLConf): RDD[InternalRow] = { + // Find the ordinals and data types of the requested columns. + val (requestedColumnIndices, requestedColumnDataTypes) = + selectedAttributes.map { a => + cacheAttributes.map(_.exprId).indexOf(a.exprId) -> a.dataType + }.unzip + + val columnTypes = requestedColumnDataTypes.map { + case udt: UserDefinedType[_] => udt.sqlType + case other => other + }.toArray + + input.mapPartitionsInternal { cachedBatchIterator => + val columnarIterator = GenerateColumnAccessor.generate(columnTypes) + columnarIterator.initialize(cachedBatchIterator.asInstanceOf[Iterator[DefaultCachedBatch]], + columnTypes, + requestedColumnIndices.toArray) + columnarIterator + } + } +} +private[sql] case class CachedRDDBuilder( - useCompression: Boolean, - batchSize: Int, + serializer: CachedBatchSerializer, storageLevel: StorageLevel, @transient cachedPlan: SparkPlan, tableName: Option[String]) { @@ -85,54 +241,24 @@ case class CachedRDDBuilder( } private def buildBuffers(): RDD[CachedBatch] = { - val output = cachedPlan.output - val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator => - new Iterator[CachedBatch] { - def next(): CachedBatch = { - val columnBuilders = output.map { attribute => - ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression) - }.toArray - - var rowCount = 0 - var totalSize = 0L - while (rowIterator.hasNext && rowCount < batchSize - && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) { - val row = rowIterator.next() - - // Added for SPARK-6082. This assertion can be useful for scenarios when something - // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM - // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat - // hard to decipher. - assert( - row.numFields == columnBuilders.length, - s"Row column number mismatch, expected ${output.size} columns, " + - s"but got ${row.numFields}." + - s"\nRow content: $row") - - var i = 0 - totalSize = 0 - while (i < row.numFields) { - columnBuilders(i).appendFrom(row, i) - totalSize += columnBuilders(i).columnStats.sizeInBytes - i += 1 - } - rowCount += 1 - } - - sizeInBytesStats.add(totalSize) - rowCountStats.add(rowCount) - - val stats = InternalRow.fromSeq( - columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq) - CachedBatch(rowCount, columnBuilders.map { builder => - JavaUtils.bufferToArray(builder.build()) - }, stats) - } - - def hasNext: Boolean = rowIterator.hasNext - } + val cb = if (cachedPlan.supportsColumnar) { + serializer.convertColumnarBatchToCachedBatch( + cachedPlan.executeColumnar(), + cachedPlan.output, + storageLevel, + cachedPlan.conf) + } else { + serializer.convertInternalRowToCachedBatch( + cachedPlan.execute(), + cachedPlan.output, + storageLevel, + cachedPlan.conf) + } + val cached = cb.map { batch => + sizeInBytesStats.add(batch.sizeInBytes) + rowCountStats.add(batch.numRows) + batch }.persist(storageLevel) - cached.setName(cachedName) cached } @@ -140,22 +266,74 @@ case class CachedRDDBuilder( object InMemoryRelation { + private[this] var ser: Option[CachedBatchSerializer] = None + private[this] def getSerializer(sqlConf: SQLConf): CachedBatchSerializer = synchronized { + if (ser.isEmpty) { + val serName = sqlConf.getConf(StaticSQLConf.SPARK_CACHE_SERIALIZER) + val serClass = Utils.classForName(serName) + val instance = serClass.getConstructor().newInstance().asInstanceOf[CachedBatchSerializer] + ser = Some(instance) + } + ser.get + } + + /* Visible for testing */ + private[columnar] def clearSerializer(): Unit = synchronized { ser = None } + + def convertToColumnarIfPossible(plan: SparkPlan): SparkPlan = plan match { + case gen: WholeStageCodegenExec => gen.child match { + case c2r: ColumnarToRowTransition => c2r.child match { + case ia: InputAdapter => ia.child + case _ => plan + } + case _ => plan + } + case c2r: ColumnarToRowTransition => // This matches when whole stage code gen is disabled. + c2r.child + case _ => plan + } + def apply( - useCompression: Boolean, - batchSize: Int, + storageLevel: StorageLevel, + qe: QueryExecution, + tableName: Option[String]): InMemoryRelation = { + val optimizedPlan = qe.optimizedPlan + val serializer = getSerializer(optimizedPlan.conf) + val child = if (serializer.supportsColumnarInput(optimizedPlan.output)) { + convertToColumnarIfPossible(qe.executedPlan) + } else { + qe.executedPlan + } + val cacheBuilder = CachedRDDBuilder(serializer, storageLevel, child, tableName) + val relation = new InMemoryRelation(child.output, cacheBuilder, optimizedPlan.outputOrdering) + relation.statsOfPlanToCache = optimizedPlan.stats + relation + } + + /** + * This API is intended only to be used for testing. + */ + def apply( + serializer: CachedBatchSerializer, storageLevel: StorageLevel, child: SparkPlan, tableName: Option[String], optimizedPlan: LogicalPlan): InMemoryRelation = { - val cacheBuilder = CachedRDDBuilder(useCompression, batchSize, storageLevel, child, tableName) + val cacheBuilder = CachedRDDBuilder(serializer, storageLevel, child, tableName) val relation = new InMemoryRelation(child.output, cacheBuilder, optimizedPlan.outputOrdering) relation.statsOfPlanToCache = optimizedPlan.stats relation } - def apply(cacheBuilder: CachedRDDBuilder, optimizedPlan: LogicalPlan): InMemoryRelation = { + def apply(cacheBuilder: CachedRDDBuilder, qe: QueryExecution): InMemoryRelation = { + val optimizedPlan = qe.optimizedPlan + val newBuilder = if (cacheBuilder.serializer.supportsColumnarInput(optimizedPlan.output)) { + cacheBuilder.copy(cachedPlan = convertToColumnarIfPossible(qe.executedPlan)) + } else { + cacheBuilder.copy(cachedPlan = qe.executedPlan) + } val relation = new InMemoryRelation( - cacheBuilder.cachedPlan.output, cacheBuilder, optimizedPlan.outputOrdering) + newBuilder.cachedPlan.output, newBuilder, optimizedPlan.outputOrdering) relation.statsOfPlanToCache = optimizedPlan.stats relation } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index f03c2586048bd..e4194562b7a5d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -17,19 +17,15 @@ package org.apache.spark.sql.execution.columnar -import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.columnar.CachedBatch import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.execution.metric.SQLMetrics -import org.apache.spark.sql.execution.vectorized._ -import org.apache.spark.sql.types._ -import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} - +import org.apache.spark.sql.vectorized.ColumnarBatch case class InMemoryTableScanExec( attributes: Seq[Attribute], @@ -57,68 +53,29 @@ case class InMemoryTableScanExec( relation = relation.canonicalized.asInstanceOf[InMemoryRelation]) override def vectorTypes: Option[Seq[String]] = - Option(Seq.fill(attributes.length)( - if (!conf.offHeapColumnVectorEnabled) { - classOf[OnHeapColumnVector].getName - } else { - classOf[OffHeapColumnVector].getName - } - )) + relation.cacheBuilder.serializer.vectorTypes(attributes, conf) /** * If true, get data from ColumnVector in ColumnarBatch, which are generally faster. * If false, get data from UnsafeRow build from CachedBatch */ override val supportsColumnar: Boolean = { - // In the initial implementation, for ease of review - // support only primitive data types and # of fields is less than wholeStageMaxNumFields - conf.cacheVectorizedReaderEnabled && relation.schema.fields.forall(f => f.dataType match { - case BooleanType | ByteType | ShortType | IntegerType | LongType | - FloatType | DoubleType => true - case _ => false - }) && !WholeStageCodegenExec.isTooManyFields(conf, relation.schema) - } - - private val columnIndices = - attributes.map(a => relation.output.map(o => o.exprId).indexOf(a.exprId)).toArray - - private val relationSchema = relation.schema.toArray - - private lazy val columnarBatchSchema = new StructType(columnIndices.map(i => relationSchema(i))) - - private def createAndDecompressColumn( - cachedColumnarBatch: CachedBatch, - offHeapColumnVectorEnabled: Boolean): ColumnarBatch = { - val rowCount = cachedColumnarBatch.numRows - val taskContext = Option(TaskContext.get()) - val columnVectors = if (!offHeapColumnVectorEnabled || taskContext.isEmpty) { - OnHeapColumnVector.allocateColumns(rowCount, columnarBatchSchema) - } else { - OffHeapColumnVector.allocateColumns(rowCount, columnarBatchSchema) - } - val columnarBatch = new ColumnarBatch(columnVectors.asInstanceOf[Array[ColumnVector]]) - columnarBatch.setNumRows(rowCount) - - for (i <- attributes.indices) { - ColumnAccessor.decompress( - cachedColumnarBatch.buffers(columnIndices(i)), - columnarBatch.column(i).asInstanceOf[WritableColumnVector], - columnarBatchSchema.fields(i).dataType, rowCount) - } - taskContext.foreach(_.addTaskCompletionListener[Unit](_ => columnarBatch.close())) - columnarBatch + conf.cacheVectorizedReaderEnabled && + !WholeStageCodegenExec.isTooManyFields(conf, relation.schema) && + relation.cacheBuilder.serializer.supportsColumnarOutput(relation.schema) } private lazy val columnarInputRDD: RDD[ColumnarBatch] = { val numOutputRows = longMetric("numOutputRows") val buffers = filteredCachedBatches() - val offHeapColumnVectorEnabled = conf.offHeapColumnVectorEnabled - buffers - .map(createAndDecompressColumn(_, offHeapColumnVectorEnabled)) - .map { buffer => - numOutputRows += buffer.numRows() - buffer - } + relation.cacheBuilder.serializer.convertCachedBatchToColumnarBatch( + buffers, + relation.output, + attributes, + conf).map { cb => + numOutputRows += cb.numRows() + cb + } } private lazy val inputRDD: RDD[InternalRow] = { @@ -130,35 +87,24 @@ case class InMemoryTableScanExec( val numOutputRows = longMetric("numOutputRows") // Using these variables here to avoid serialization of entire objects (if referenced // directly) within the map Partitions closure. - val relOutput: AttributeSeq = relation.output - - filteredCachedBatches().mapPartitionsInternal { cachedBatchIterator => - // Find the ordinals and data types of the requested columns. - val (requestedColumnIndices, requestedColumnDataTypes) = - attributes.map { a => - relOutput.indexOf(a.exprId) -> a.dataType - }.unzip - - // update SQL metrics - val withMetrics = cachedBatchIterator.map { batch => - if (enableAccumulatorsForTest) { - readBatches.add(1) + val relOutput = relation.output + val serializer = relation.cacheBuilder.serializer + + // update SQL metrics + val withMetrics = + filteredCachedBatches().mapPartitionsInternal { iter => + if (enableAccumulatorsForTest && iter.hasNext) { + readPartitions.add(1) + } + iter.map { batch => + if (enableAccumulatorsForTest) { + readBatches.add(1) + } + numOutputRows += batch.numRows + batch } - numOutputRows += batch.numRows - batch - } - - val columnTypes = requestedColumnDataTypes.map { - case udt: UserDefinedType[_] => udt.sqlType - case other => other - }.toArray - val columnarIterator = GenerateColumnAccessor.generate(columnTypes) - columnarIterator.initialize(withMetrics, columnTypes, requestedColumnIndices.toArray) - if (enableAccumulatorsForTest && columnarIterator.hasNext) { - readPartitions.add(1) } - columnarIterator - } + serializer.convertCachedBatchToInternalRow(withMetrics, relOutput, attributes, conf) } override def output: Seq[Attribute] = attributes @@ -186,114 +132,6 @@ case class InMemoryTableScanExec( override def outputOrdering: Seq[SortOrder] = relation.cachedPlan.outputOrdering.map(updateAttribute(_).asInstanceOf[SortOrder]) - // Keeps relation's partition statistics because we don't serialize relation. - private val stats = relation.partitionStatistics - private def statsFor(a: Attribute) = stats.forAttribute(a) - - // Currently, only use statistics from atomic types except binary type only. - private object ExtractableLiteral { - def unapply(expr: Expression): Option[Literal] = expr match { - case lit: Literal => lit.dataType match { - case BinaryType => None - case _: AtomicType => Some(lit) - case _ => None - } - case _ => None - } - } - - // Returned filter predicate should return false iff it is impossible for the input expression - // to evaluate to `true` based on statistics collected about this partition batch. - @transient lazy val buildFilter: PartialFunction[Expression, Expression] = { - case And(lhs: Expression, rhs: Expression) - if buildFilter.isDefinedAt(lhs) || buildFilter.isDefinedAt(rhs) => - (buildFilter.lift(lhs) ++ buildFilter.lift(rhs)).reduce(_ && _) - - case Or(lhs: Expression, rhs: Expression) - if buildFilter.isDefinedAt(lhs) && buildFilter.isDefinedAt(rhs) => - buildFilter(lhs) || buildFilter(rhs) - - case EqualTo(a: AttributeReference, ExtractableLiteral(l)) => - statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound - case EqualTo(ExtractableLiteral(l), a: AttributeReference) => - statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound - - case EqualNullSafe(a: AttributeReference, ExtractableLiteral(l)) => - statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound - case EqualNullSafe(ExtractableLiteral(l), a: AttributeReference) => - statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound - - case LessThan(a: AttributeReference, ExtractableLiteral(l)) => statsFor(a).lowerBound < l - case LessThan(ExtractableLiteral(l), a: AttributeReference) => l < statsFor(a).upperBound - - case LessThanOrEqual(a: AttributeReference, ExtractableLiteral(l)) => - statsFor(a).lowerBound <= l - case LessThanOrEqual(ExtractableLiteral(l), a: AttributeReference) => - l <= statsFor(a).upperBound - - case GreaterThan(a: AttributeReference, ExtractableLiteral(l)) => l < statsFor(a).upperBound - case GreaterThan(ExtractableLiteral(l), a: AttributeReference) => statsFor(a).lowerBound < l - - case GreaterThanOrEqual(a: AttributeReference, ExtractableLiteral(l)) => - l <= statsFor(a).upperBound - case GreaterThanOrEqual(ExtractableLiteral(l), a: AttributeReference) => - statsFor(a).lowerBound <= l - - case IsNull(a: Attribute) => statsFor(a).nullCount > 0 - case IsNotNull(a: Attribute) => statsFor(a).count - statsFor(a).nullCount > 0 - - case In(a: AttributeReference, list: Seq[Expression]) - if list.forall(ExtractableLiteral.unapply(_).isDefined) && list.nonEmpty => - list.map(l => statsFor(a).lowerBound <= l.asInstanceOf[Literal] && - l.asInstanceOf[Literal] <= statsFor(a).upperBound).reduce(_ || _) - - // This is an example to explain how it works, imagine that the id column stored as follows: - // __________________________________________ - // | Partition ID | lowerBound | upperBound | - // |--------------|------------|------------| - // | p1 | '1' | '9' | - // | p2 | '10' | '19' | - // | p3 | '20' | '29' | - // | p4 | '30' | '39' | - // | p5 | '40' | '49' | - // |______________|____________|____________| - // - // A filter: df.filter($"id".startsWith("2")). - // In this case it substr lowerBound and upperBound: - // ________________________________________________________________________________________ - // | Partition ID | lowerBound.substr(0, Length("2")) | upperBound.substr(0, Length("2")) | - // |--------------|-----------------------------------|-----------------------------------| - // | p1 | '1' | '9' | - // | p2 | '1' | '1' | - // | p3 | '2' | '2' | - // | p4 | '3' | '3' | - // | p5 | '4' | '4' | - // |______________|___________________________________|___________________________________| - // - // We can see that we only need to read p1 and p3. - case StartsWith(a: AttributeReference, ExtractableLiteral(l)) => - statsFor(a).lowerBound.substr(0, Length(l)) <= l && - l <= statsFor(a).upperBound.substr(0, Length(l)) - } - - lazy val partitionFilters: Seq[Expression] = { - predicates.flatMap { p => - val filter = buildFilter.lift(p) - val boundFilter = - filter.map( - BindReferences.bindReference( - _, - stats.schema, - allowFailures = true)) - - boundFilter.foreach(_ => - filter.foreach(f => logInfo(s"Predicate $p generates partition filter: $f"))) - - // If the filter can't be resolved then we are missing required statistics. - boundFilter.filter(_.resolved) - } - } - lazy val enableAccumulatorsForTest: Boolean = sqlContext.conf.inMemoryTableScanStatisticsEnabled // Accumulators used for testing purposes @@ -303,37 +141,13 @@ case class InMemoryTableScanExec( private val inMemoryPartitionPruningEnabled = sqlContext.conf.inMemoryPartitionPruning private def filteredCachedBatches(): RDD[CachedBatch] = { - // Using these variables here to avoid serialization of entire objects (if referenced directly) - // within the map Partitions closure. - val schema = stats.schema - val schemaIndex = schema.zipWithIndex val buffers = relation.cacheBuilder.cachedColumnBuffers - buffers.mapPartitionsWithIndexInternal { (index, cachedBatchIterator) => - val partitionFilter = Predicate.create( - partitionFilters.reduceOption(And).getOrElse(Literal(true)), - schema) - partitionFilter.initialize(index) - - // Do partition batch pruning if enabled - if (inMemoryPartitionPruningEnabled) { - cachedBatchIterator.filter { cachedBatch => - if (!partitionFilter.eval(cachedBatch.stats)) { - logDebug { - val statsString = schemaIndex.map { case (a, i) => - val value = cachedBatch.stats.get(i, a.dataType) - s"${a.name}: $value" - }.mkString(", ") - s"Skipping partition based on stats $statsString" - } - false - } else { - true - } - } - } else { - cachedBatchIterator - } + if (inMemoryPartitionPruningEnabled) { + val filterFunc = relation.cacheBuilder.serializer.buildFilter(predicates, relation.output) + buffers.mapPartitionsWithIndexInternal(filterFunc) + } else { + buffers } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index 3c6649b26ecd2..816f5f45860c6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -819,8 +819,10 @@ object JdbcUtils extends Logging { if (null != customSchema && customSchema.nonEmpty) { val userSchema = CatalystSqlParser.parseTableSchema(customSchema) - SchemaUtils.checkColumnNameDuplication( - userSchema.map(_.name), "in the customSchema option value", nameEquality) + SchemaUtils.checkSchemaColumnNameDuplication( + userSchema, + "in the customSchema option value", + nameEquality) // This is resolved by names, use the custom filed dataType to replace the default dataType. val newSchema = tableSchema.map { col => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala index df3f231f7d0ef..6dda1d4aaf37e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala @@ -126,11 +126,18 @@ class V2SessionCatalog(catalog: SessionCatalog, conf: SQLConf) val schema = CatalogV2Util.applySchemaChanges(catalogTable.schema, changes) val comment = properties.get(TableCatalog.PROP_COMMENT) val owner = properties.getOrElse(TableCatalog.PROP_OWNER, catalogTable.owner) + val location = properties.get(TableCatalog.PROP_LOCATION).map(CatalogUtils.stringToURI) + val storage = if (location.isDefined) { + catalogTable.storage.copy(locationUri = location) + } else { + catalogTable.storage + } try { catalog.alterTable( - catalogTable - .copy(properties = properties, schema = schema, owner = owner, comment = comment)) + catalogTable.copy( + properties = properties, schema = schema, owner = owner, comment = comment, + storage = storage)) } catch { case _: NoSuchTableException => throw new NoSuchTableException(ident) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index fe0057c3d588b..653fe5bf7c9b3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -2478,6 +2478,8 @@ object functions { /** * Extract a specific group matched by a Java regex, from the specified string column. * If the regex did not match, or the specified group did not match, an empty string is returned. + * if the specified group index exceeds the group count of regex, an IllegalArgumentException + * will be thrown. * * @group string_funcs * @since 1.5.0 @@ -2486,6 +2488,19 @@ object functions { RegExpExtract(e.expr, lit(exp).expr, lit(groupIdx).expr) } + /** + * Extract all specific groups matched by a Java regex, from the specified string column. + * If the regex did not match, or the specified group did not match, return an empty array. + * if the specified group index exceeds the group count of regex, an IllegalArgumentException + * will be thrown. + * + * @group string_funcs + * @since 3.1.0 + */ + def regexp_extract_all(e: Column, exp: String, groupIdx: Int): Column = withExpr { + RegExpExtractAll(e.expr, lit(exp).expr, lit(groupIdx).expr) + } + /** * Replace all substrings of the specified string value that match regexp with rep. * diff --git a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md index edf2ede9e5a44..a212d8ce40642 100644 --- a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md +++ b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md @@ -214,6 +214,7 @@ | org.apache.spark.sql.catalyst.expressions.Randn | randn | SELECT randn() | struct | | org.apache.spark.sql.catalyst.expressions.Rank | rank | N/A | N/A | | org.apache.spark.sql.catalyst.expressions.RegExpExtract | regexp_extract | SELECT regexp_extract('100-200', '(\\d+)-(\\d+)', 1) | struct | +| org.apache.spark.sql.catalyst.expressions.RegExpExtractAll | regexp_extract_all | SELECT regexp_extract_all('100-200, 300-400', '(\\d+)-(\\d+)', 1) | struct> | | org.apache.spark.sql.catalyst.expressions.RegExpReplace | regexp_replace | SELECT regexp_replace('100-200', '(\\d+)', 'num') | struct | | org.apache.spark.sql.catalyst.expressions.Remainder | % | SELECT 2 % 1.8 | struct<(CAST(CAST(2 AS DECIMAL(1,0)) AS DECIMAL(2,1)) % CAST(1.8 AS DECIMAL(2,1))):decimal(2,1)> | | org.apache.spark.sql.catalyst.expressions.Remainder | mod | SELECT 2 % 1.8 | struct<(CAST(CAST(2 AS DECIMAL(1,0)) AS DECIMAL(2,1)) % CAST(1.8 AS DECIMAL(2,1))):decimal(2,1)> | diff --git a/sql/core/src/test/resources/sql-tests/inputs/group-by-filter.sql b/sql/core/src/test/resources/sql-tests/inputs/group-by-filter.sql index 4c1816e93b083..24d303621faea 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/group-by-filter.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/group-by-filter.sql @@ -36,8 +36,13 @@ SELECT COUNT(id) FILTER (WHERE hiredate = date "2001-01-01") FROM emp; SELECT COUNT(id) FILTER (WHERE hiredate = to_date('2001-01-01 00:00:00')) FROM emp; SELECT COUNT(id) FILTER (WHERE hiredate = to_timestamp("2001-01-01 00:00:00")) FROM emp; SELECT COUNT(id) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd") = "2001-01-01") FROM emp; --- [SPARK-30276] Support Filter expression allows simultaneous use of DISTINCT --- SELECT COUNT(DISTINCT id) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") = "2001-01-01 00:00:00") FROM emp; +SELECT COUNT(DISTINCT id) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") = "2001-01-01 00:00:00") FROM emp; +SELECT COUNT(DISTINCT id), COUNT(DISTINCT id) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") = "2001-01-01 00:00:00") FROM emp; +SELECT COUNT(DISTINCT id) FILTER (WHERE hiredate = to_timestamp("2001-01-01 00:00:00")), COUNT(DISTINCT id) FILTER (WHERE hiredate = to_date('2001-01-01 00:00:00')) FROM emp; +SELECT SUM(salary), COUNT(DISTINCT id), COUNT(DISTINCT id) FILTER (WHERE hiredate = date "2001-01-01") FROM emp; +SELECT COUNT(DISTINCT 1) FILTER (WHERE a = 1) FROM testData; +SELECT COUNT(DISTINCT id) FILTER (WHERE true) FROM emp; +SELECT COUNT(DISTINCT id) FILTER (WHERE false) FROM emp; -- Aggregate with filter and non-empty GroupBy expressions. SELECT a, COUNT(b) FILTER (WHERE a >= 2) FROM testData GROUP BY a; @@ -47,8 +52,11 @@ SELECT dept_id, SUM(salary) FILTER (WHERE hiredate > date "2003-01-01") FROM emp SELECT dept_id, SUM(salary) FILTER (WHERE hiredate > to_date("2003-01-01")) FROM emp GROUP BY dept_id; SELECT dept_id, SUM(salary) FILTER (WHERE hiredate > to_timestamp("2003-01-01 00:00:00")) FROM emp GROUP BY dept_id; SELECT dept_id, SUM(salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd") > "2003-01-01") FROM emp GROUP BY dept_id; --- [SPARK-30276] Support Filter expression allows simultaneous use of DISTINCT --- SELECT dept_id, SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") > "2001-01-01 00:00:00") FROM emp GROUP BY dept_id; +SELECT dept_id, SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") > "2001-01-01 00:00:00") FROM emp GROUP BY dept_id; +SELECT dept_id, SUM(DISTINCT salary), SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") > "2001-01-01 00:00:00") FROM emp GROUP BY dept_id; +SELECT dept_id, SUM(DISTINCT salary) FILTER (WHERE hiredate > date "2001-01-01"), SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") > "2001-01-01 00:00:00") FROM emp GROUP BY dept_id; +SELECT dept_id, COUNT(id), SUM(DISTINCT salary), SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd") > "2001-01-01") FROM emp GROUP BY dept_id; +SELECT b, COUNT(DISTINCT 1) FILTER (WHERE a = 1) FROM testData GROUP BY b; -- Aggregate with filter and grouped by literals. SELECT 'foo', COUNT(a) FILTER (WHERE b <= 2) FROM testData GROUP BY 1; @@ -61,13 +69,24 @@ select dept_id, count(distinct emp_name), count(distinct hiredate), sum(salary), select dept_id, count(distinct emp_name), count(distinct hiredate), sum(salary), sum(salary) filter (where id + dept_id > 500) from emp group by dept_id; select dept_id, count(distinct emp_name), count(distinct hiredate), sum(salary) filter (where salary < 400.00D), sum(salary) filter (where id > 200) from emp group by dept_id; select dept_id, count(distinct emp_name), count(distinct hiredate), sum(salary) filter (where salary < 400.00D), sum(salary) filter (where id + dept_id > 500) from emp group by dept_id; --- [SPARK-30276] Support Filter expression allows simultaneous use of DISTINCT --- select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate), sum(salary) from emp group by dept_id; --- select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id; --- select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D) from emp group by dept_id; --- select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D), sum(salary) filter (where id > 200) from emp group by dept_id; --- select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct emp_name), sum(salary) from emp group by dept_id; --- select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct emp_name) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id; +select dept_id, count(distinct emp_name) filter (where id > 200), sum(salary) from emp group by dept_id; +select dept_id, count(distinct emp_name) filter (where id + dept_id > 500), sum(salary) from emp group by dept_id; +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id > 200), sum(salary) from emp group by dept_id; +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id + dept_id > 500), sum(salary) from emp group by dept_id; +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id > 200), sum(salary), sum(salary) filter (where id > 200) from emp group by dept_id; +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id + dept_id > 500), sum(salary), sum(salary) filter (where id > 200) from emp group by dept_id; +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate), sum(salary) from emp group by dept_id; +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id; +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D) from emp group by dept_id; +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D), sum(salary) filter (where id > 200) from emp group by dept_id; +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct emp_name), sum(salary) from emp group by dept_id; +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct emp_name) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id; +select dept_id, sum(distinct (id + dept_id)) filter (where id > 200), count(distinct hiredate), sum(salary) from emp group by dept_id; +select dept_id, sum(distinct (id + dept_id)) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id; +select dept_id, avg(distinct (id + dept_id)) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D) from emp group by dept_id; +select dept_id, count(distinct emp_name, hiredate) filter (where id > 200), sum(salary) from emp group by dept_id; +select dept_id, count(distinct emp_name, hiredate) filter (where id > 0), sum(salary) from emp group by dept_id; +select dept_id, count(distinct 1), count(distinct 1) filter (where id > 200), sum(salary) from emp group by dept_id; -- Aggregate with filter and grouped by literals (hash aggregate), here the input table is filtered using WHERE. SELECT 'foo', APPROX_COUNT_DISTINCT(a) FILTER (WHERE b >= 0) FROM testData WHERE a = 0 GROUP BY 1; @@ -81,9 +100,8 @@ SELECT a + 2, COUNT(b) FILTER (WHERE b IN (1, 2)) FROM testData GROUP BY a + 1; SELECT a + 1 + 1, COUNT(b) FILTER (WHERE b > 0) FROM testData GROUP BY a + 1; -- Aggregate with filter, foldable input and multiple distinct groups. --- [SPARK-30276] Support Filter expression allows simultaneous use of DISTINCT --- SELECT COUNT(DISTINCT b) FILTER (WHERE b > 0), COUNT(DISTINCT b, c) FILTER (WHERE b > 0 AND c > 2) --- FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a; +SELECT COUNT(DISTINCT b) FILTER (WHERE b > 0), COUNT(DISTINCT b, c) FILTER (WHERE b > 0 AND c > 2) +FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a; -- Check analysis exceptions SELECT a AS k, COUNT(b) FILTER (WHERE b > 0) FROM testData GROUP BY k; diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part3.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part3.sql index 746b677234832..657ea59ec8f11 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part3.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part3.sql @@ -241,10 +241,9 @@ select sum(1/ten) filter (where ten > 0) from tenk1; -- select ten, sum(distinct four) filter (where four::text ~ '123') from onek a -- group by ten; --- [SPARK-30276] Support Filter expression allows simultaneous use of DISTINCT --- select ten, sum(distinct four) filter (where four > 10) from onek a --- group by ten --- having exists (select 1 from onek b where sum(distinct a.four) = b.four); +select ten, sum(distinct four) filter (where four > 10) from onek a +group by ten +having exists (select 1 from onek b where sum(distinct a.four) = b.four); -- [SPARK-28682] ANSI SQL: Collation Support -- select max(foo COLLATE "C") filter (where (bar collate "POSIX") > '0') diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/groupingsets.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/groupingsets.sql index fc54d179f742c..45617c53166aa 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/groupingsets.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/groupingsets.sql @@ -336,9 +336,8 @@ order by 2,1; -- order by 2,1; -- FILTER queries --- [SPARK-30276] Support Filter expression allows simultaneous use of DISTINCT --- select ten, sum(distinct four) filter (where string(four) like '123') from onek a --- group by rollup(ten); +select ten, sum(distinct four) filter (where string(four) like '123') from onek a +group by rollup(ten); -- More rescan tests -- [SPARK-27877] ANSI SQL: LATERAL derived table(T491) diff --git a/sql/core/src/test/resources/sql-tests/inputs/regexp-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/regexp-functions.sql index 8a531be30d896..7128dee0a00d7 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/regexp-functions.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/regexp-functions.sql @@ -4,8 +4,30 @@ SELECT regexp_extract('1a 2b 14m', '\\d+', 0); SELECT regexp_extract('1a 2b 14m', '\\d+', 1); SELECT regexp_extract('1a 2b 14m', '\\d+', 2); SELECT regexp_extract('1a 2b 14m', '\\d+', -1); +SELECT regexp_extract('1a 2b 14m', '(\\d+)?', 1); +SELECT regexp_extract('a b m', '(\\d+)?', 1); SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)'); SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 0); SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 1); SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 2); +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 3); SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', -1); +SELECT regexp_extract('1a 2b 14m', '(\\d+)?([a-z]+)', 1); +SELECT regexp_extract('a b m', '(\\d+)?([a-z]+)', 1); + +-- regexp_extract_all +SELECT regexp_extract_all('1a 2b 14m', '\\d+'); +SELECT regexp_extract_all('1a 2b 14m', '\\d+', 0); +SELECT regexp_extract_all('1a 2b 14m', '\\d+', 1); +SELECT regexp_extract_all('1a 2b 14m', '\\d+', 2); +SELECT regexp_extract_all('1a 2b 14m', '\\d+', -1); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)?', 1); +SELECT regexp_extract_all('a 2b 14m', '(\\d+)?', 1); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)'); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 0); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 1); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 2); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 3); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', -1); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)?([a-z]+)', 1); +SELECT regexp_extract_all('a 2b 14m', '(\\d+)?([a-z]+)', 1); diff --git a/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out index d41d25280146b..c349d9d84c226 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 37 +-- Number of queries: 68 -- !query @@ -94,6 +94,62 @@ struct +-- !query output +2 + + +-- !query +SELECT COUNT(DISTINCT id), COUNT(DISTINCT id) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") = "2001-01-01 00:00:00") FROM emp +-- !query schema +struct +-- !query output +8 2 + + +-- !query +SELECT COUNT(DISTINCT id) FILTER (WHERE hiredate = to_timestamp("2001-01-01 00:00:00")), COUNT(DISTINCT id) FILTER (WHERE hiredate = to_date('2001-01-01 00:00:00')) FROM emp +-- !query schema +struct +-- !query output +2 2 + + +-- !query +SELECT SUM(salary), COUNT(DISTINCT id), COUNT(DISTINCT id) FILTER (WHERE hiredate = date "2001-01-01") FROM emp +-- !query schema +struct +-- !query output +2450.0 8 2 + + +-- !query +SELECT COUNT(DISTINCT 1) FILTER (WHERE a = 1) FROM testData +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT COUNT(DISTINCT id) FILTER (WHERE true) FROM emp +-- !query schema +struct +-- !query output +8 + + +-- !query +SELECT COUNT(DISTINCT id) FILTER (WHERE false) FROM emp +-- !query schema +struct +-- !query output +0 + + -- !query SELECT a, COUNT(b) FILTER (WHERE a >= 2) FROM testData GROUP BY a -- !query schema @@ -177,6 +233,68 @@ struct "2001-01-01 00:00:00") FROM emp GROUP BY dept_id +-- !query schema +struct 2001-01-01 00:00:00)):double> +-- !query output +10 300.0 +100 400.0 +20 300.0 +30 400.0 +70 150.0 +NULL NULL + + +-- !query +SELECT dept_id, SUM(DISTINCT salary), SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") > "2001-01-01 00:00:00") FROM emp GROUP BY dept_id +-- !query schema +struct 2001-01-01 00:00:00)):double> +-- !query output +10 300.0 300.0 +100 400.0 400.0 +20 300.0 300.0 +30 400.0 400.0 +70 150.0 150.0 +NULL 400.0 NULL + + +-- !query +SELECT dept_id, SUM(DISTINCT salary) FILTER (WHERE hiredate > date "2001-01-01"), SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") > "2001-01-01 00:00:00") FROM emp GROUP BY dept_id +-- !query schema +struct DATE '2001-01-01')):double,sum(DISTINCT salary) FILTER (WHERE (date_format(CAST(hiredate AS TIMESTAMP), yyyy-MM-dd HH:mm:ss) > 2001-01-01 00:00:00)):double> +-- !query output +10 300.0 300.0 +100 400.0 400.0 +20 300.0 300.0 +30 400.0 400.0 +70 150.0 150.0 +NULL NULL NULL + + +-- !query +SELECT dept_id, COUNT(id), SUM(DISTINCT salary), SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd") > "2001-01-01") FROM emp GROUP BY dept_id +-- !query schema +struct 2001-01-01)):double> +-- !query output +10 3 300.0 300.0 +100 2 400.0 400.0 +20 1 300.0 300.0 +30 1 400.0 400.0 +70 1 150.0 150.0 +NULL 1 400.0 NULL + + +-- !query +SELECT b, COUNT(DISTINCT 1) FILTER (WHERE a = 1) FROM testData GROUP BY b +-- !query schema +struct +-- !query output +1 1 +2 1 +NULL 0 + + -- !query SELECT 'foo', COUNT(a) FILTER (WHERE b <= 2) FROM testData GROUP BY 1 -- !query schema @@ -261,6 +379,240 @@ struct 200), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,sum(salary):double> +-- !query output +10 0 400.0 +100 2 800.0 +20 1 300.0 +30 1 400.0 +70 1 150.0 +NULL 1 400.0 + + +-- !query +select dept_id, count(distinct emp_name) filter (where id + dept_id > 500), sum(salary) from emp group by dept_id +-- !query schema +struct 500)):bigint,sum(salary):double> +-- !query output +10 0 400.0 +100 2 800.0 +20 0 300.0 +30 0 400.0 +70 1 150.0 +NULL 0 400.0 + + +-- !query +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id > 200), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,sum(salary):double> +-- !query output +10 2 0 400.0 +100 2 2 800.0 +20 1 1 300.0 +30 1 1 400.0 +70 1 1 150.0 +NULL 1 1 400.0 + + +-- !query +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id + dept_id > 500), sum(salary) from emp group by dept_id +-- !query schema +struct 500)):bigint,sum(salary):double> +-- !query output +10 2 0 400.0 +100 2 2 800.0 +20 1 0 300.0 +30 1 0 400.0 +70 1 1 150.0 +NULL 1 0 400.0 + + +-- !query +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id > 200), sum(salary), sum(salary) filter (where id > 200) from emp group by dept_id +-- !query schema +struct 200)):bigint,sum(salary):double,sum(salary) FILTER (WHERE (id > 200)):double> +-- !query output +10 2 0 400.0 NULL +100 2 2 800.0 800.0 +20 1 1 300.0 300.0 +30 1 1 400.0 400.0 +70 1 1 150.0 150.0 +NULL 1 1 400.0 400.0 + + +-- !query +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id + dept_id > 500), sum(salary), sum(salary) filter (where id > 200) from emp group by dept_id +-- !query schema +struct 500)):bigint,sum(salary):double,sum(salary) FILTER (WHERE (id > 200)):double> +-- !query output +10 2 0 400.0 NULL +100 2 2 800.0 800.0 +20 1 0 300.0 300.0 +30 1 0 400.0 400.0 +70 1 1 150.0 150.0 +NULL 1 0 400.0 400.0 + + +-- !query +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,count(DISTINCT hiredate):bigint,sum(salary):double> +-- !query output +10 0 2 400.0 +100 2 2 800.0 +20 1 1 300.0 +30 1 1 400.0 +70 1 1 150.0 +NULL 1 1 400.0 + + +-- !query +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,count(DISTINCT hiredate) FILTER (WHERE (hiredate > DATE '2003-01-01')):bigint,sum(salary):double> +-- !query output +10 0 1 400.0 +100 2 1 800.0 +20 1 0 300.0 +30 1 1 400.0 +70 1 1 150.0 +NULL 1 0 400.0 + + +-- !query +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D) from emp group by dept_id +-- !query schema +struct 200)):bigint,count(DISTINCT hiredate) FILTER (WHERE (hiredate > DATE '2003-01-01')):bigint,sum(salary) FILTER (WHERE (salary < 400.0)):double> +-- !query output +10 0 1 400.0 +100 2 1 NULL +20 1 0 300.0 +30 1 1 NULL +70 1 1 150.0 +NULL 1 0 NULL + + +-- !query +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D), sum(salary) filter (where id > 200) from emp group by dept_id +-- !query schema +struct 200)):bigint,count(DISTINCT hiredate) FILTER (WHERE (hiredate > DATE '2003-01-01')):bigint,sum(salary) FILTER (WHERE (salary < 400.0)):double,sum(salary) FILTER (WHERE (id > 200)):double> +-- !query output +10 0 1 400.0 NULL +100 2 1 NULL 800.0 +20 1 0 300.0 300.0 +30 1 1 NULL 400.0 +70 1 1 150.0 150.0 +NULL 1 0 NULL 400.0 + + +-- !query +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct emp_name), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,count(DISTINCT emp_name):bigint,sum(salary):double> +-- !query output +10 0 2 400.0 +100 2 2 800.0 +20 1 1 300.0 +30 1 1 400.0 +70 1 1 150.0 +NULL 1 1 400.0 + + +-- !query +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct emp_name) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,count(DISTINCT emp_name) FILTER (WHERE (hiredate > DATE '2003-01-01')):bigint,sum(salary):double> +-- !query output +10 0 1 400.0 +100 2 1 800.0 +20 1 0 300.0 +30 1 1 400.0 +70 1 1 150.0 +NULL 1 0 400.0 + + +-- !query +select dept_id, sum(distinct (id + dept_id)) filter (where id > 200), count(distinct hiredate), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,count(DISTINCT hiredate):bigint,sum(salary):double> +-- !query output +10 NULL 2 400.0 +100 1500 2 800.0 +20 320 1 300.0 +30 430 1 400.0 +70 870 1 150.0 +NULL NULL 1 400.0 + + +-- !query +select dept_id, sum(distinct (id + dept_id)) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,count(DISTINCT hiredate) FILTER (WHERE (hiredate > DATE '2003-01-01')):bigint,sum(salary):double> +-- !query output +10 NULL 1 400.0 +100 1500 1 800.0 +20 320 0 300.0 +30 430 1 400.0 +70 870 1 150.0 +NULL NULL 0 400.0 + + +-- !query +select dept_id, avg(distinct (id + dept_id)) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D) from emp group by dept_id +-- !query schema +struct 200)):double,count(DISTINCT hiredate) FILTER (WHERE (hiredate > DATE '2003-01-01')):bigint,sum(salary) FILTER (WHERE (salary < 400.0)):double> +-- !query output +10 NULL 1 400.0 +100 750.0 1 NULL +20 320.0 0 300.0 +30 430.0 1 NULL +70 870.0 1 150.0 +NULL NULL 0 NULL + + +-- !query +select dept_id, count(distinct emp_name, hiredate) filter (where id > 200), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,sum(salary):double> +-- !query output +10 0 400.0 +100 2 800.0 +20 1 300.0 +30 1 400.0 +70 1 150.0 +NULL 1 400.0 + + +-- !query +select dept_id, count(distinct emp_name, hiredate) filter (where id > 0), sum(salary) from emp group by dept_id +-- !query schema +struct 0)):bigint,sum(salary):double> +-- !query output +10 2 400.0 +100 2 800.0 +20 1 300.0 +30 1 400.0 +70 1 150.0 +NULL 1 400.0 + + +-- !query +select dept_id, count(distinct 1), count(distinct 1) filter (where id > 200), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,sum(salary):double> +-- !query output +10 1 0 400.0 +100 1 1 800.0 +20 1 1 300.0 +30 1 1 400.0 +70 1 1 150.0 +NULL 1 1 400.0 + + -- !query SELECT 'foo', APPROX_COUNT_DISTINCT(a) FILTER (WHERE b >= 0) FROM testData WHERE a = 0 GROUP BY 1 -- !query schema @@ -309,6 +661,15 @@ struct<((a + 1) + 1):int,count(b) FILTER (WHERE (b > 0)):bigint> NULL 1 +-- !query +SELECT COUNT(DISTINCT b) FILTER (WHERE b > 0), COUNT(DISTINCT b, c) FILTER (WHERE b > 0 AND c > 2) +FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a +-- !query schema +struct 0)):bigint,count(DISTINCT b, c) FILTER (WHERE ((b > 0) AND (c > 2))):bigint> +-- !query output +1 1 + + -- !query SELECT a AS k, COUNT(b) FILTER (WHERE b > 0) FROM testData GROUP BY k -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/pivot.sql.out b/sql/core/src/test/resources/sql-tests/results/pivot.sql.out index ac4e71e244bc0..bb0d452fa04a1 100644 --- a/sql/core/src/test/resources/sql-tests/results/pivot.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pivot.sql.out @@ -276,7 +276,7 @@ PIVOT ( FOR (course, year) IN (('dotNET', 2012), ('Java', 2013)) ) -- !query schema -struct +struct -- !query output 1 15000 NULL 2 NULL 30000 @@ -370,7 +370,7 @@ PIVOT ( FOR (y, course) IN ((2012, 'dotNET'), (2013, 'Java')) ) -- !query schema -struct,[2013, Java]:array> +struct,{2013, Java}:array> -- !query output 2012 [1,1] NULL 2013 NULL [2,2] @@ -404,7 +404,7 @@ PIVOT ( FOR (course, a) IN (('dotNET', array(1, 1)), ('Java', array(2, 2))) ) -- !query schema -struct +struct -- !query output 2012 15000 NULL 2013 NULL 30000 @@ -421,7 +421,7 @@ PIVOT ( FOR s IN ((1, 'a'), (2, 'b')) ) -- !query schema -struct +struct -- !query output 2012 35000 NULL 2013 NULL 78000 @@ -438,7 +438,7 @@ PIVOT ( FOR (course, s) IN (('dotNET', (1, 'a')), ('Java', (2, 'b'))) ) -- !query schema -struct +struct -- !query output 2012 15000 NULL 2013 NULL 30000 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part3.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part3.sql.out index 69f96b02782e3..e1f735e5fe1dc 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part3.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part3.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 4 +-- Number of queries: 5 -- !query @@ -27,6 +27,20 @@ struct 0)):d 2828.9682539682954 +-- !query +select ten, sum(distinct four) filter (where four > 10) from onek a +group by ten +having exists (select 1 from onek b where sum(distinct a.four) = b.four) +-- !query schema +struct 10)):bigint> +-- !query output +0 NULL +2 NULL +4 NULL +6 NULL +8 NULL + + -- !query select (select count(*) from (values (1)) t0(inner_c)) diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/groupingsets.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/groupingsets.sql.out index 7312c20876296..2619634d7d569 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/groupingsets.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/groupingsets.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 54 +-- Number of queries: 55 -- !query @@ -443,6 +443,25 @@ struct NULL 1 +-- !query +select ten, sum(distinct four) filter (where string(four) like '123') from onek a +group by rollup(ten) +-- !query schema +struct +-- !query output +0 NULL +1 NULL +2 NULL +3 NULL +4 NULL +5 NULL +6 NULL +7 NULL +8 NULL +9 NULL +NULL NULL + + -- !query select count(*) from gstest4 group by rollup(unhashable_col,unsortable_col) -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out index 64aa6053d8d70..2eef926f63e37 100644 --- a/sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 10 +-- Number of queries: 30 -- !query @@ -46,6 +46,22 @@ java.lang.IllegalArgumentException The specified group index cannot be less than zero +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)?', 1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT regexp_extract('a b m', '(\\d+)?', 1) +-- !query schema +struct +-- !query output + + + -- !query SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)') -- !query schema @@ -78,10 +94,161 @@ struct a +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 3) +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Regex group count is 2, but the specified group index is 3 + + -- !query SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', -1) -- !query schema struct<> -- !query output java.lang.IllegalArgumentException -The specified group index cannot be less than zero \ No newline at end of file +The specified group index cannot be less than zero + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)?([a-z]+)', 1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT regexp_extract('a b m', '(\\d+)?([a-z]+)', 1) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '\\d+') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Regex group count is 0, but the specified group index is 1 + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '\\d+', 0) +-- !query schema +struct> +-- !query output +["1","2","14"] + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '\\d+', 1) +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Regex group count is 0, but the specified group index is 1 + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '\\d+', 2) +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Regex group count is 0, but the specified group index is 2 + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '\\d+', -1) +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +The specified group index cannot be less than zero + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)?', 1) +-- !query schema +struct> +-- !query output +["1","","","2","","","14","",""] + + +-- !query +SELECT regexp_extract_all('a 2b 14m', '(\\d+)?', 1) +-- !query schema +struct> +-- !query output +["","","2","","","14","",""] + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)') +-- !query schema +struct> +-- !query output +["1","2","14"] + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 0) +-- !query schema +struct> +-- !query output +["1a","2b","14m"] + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 1) +-- !query schema +struct> +-- !query output +["1","2","14"] + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 2) +-- !query schema +struct> +-- !query output +["a","b","m"] + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 3) +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Regex group count is 2, but the specified group index is 3 + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', -1) +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +The specified group index cannot be less than zero + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)?([a-z]+)', 1) +-- !query schema +struct> +-- !query output +["1","2","14"] + + +-- !query +SELECT regexp_extract_all('a 2b 14m', '(\\d+)?([a-z]+)', 1) +-- !query schema +struct> +-- !query output +["","2","14"] diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out index 087b4ed9302d8..414435e6b781d 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out @@ -276,7 +276,7 @@ PIVOT ( FOR (course, year) IN (('dotNET', 2012), ('Java', 2013)) ) -- !query schema -struct +struct -- !query output 1 15000 NULL 2 NULL 30000 @@ -370,7 +370,7 @@ PIVOT ( FOR (course, a) IN (('dotNET', array(1, 1)), ('Java', array(2, 2))) ) -- !query schema -struct +struct -- !query output 2012 15000 NULL 2013 NULL 30000 @@ -387,7 +387,7 @@ PIVOT ( FOR s IN ((1, 'a'), (2, 'b')) ) -- !query schema -struct +struct -- !query output 2012 35000 NULL 2013 NULL 78000 @@ -404,7 +404,7 @@ PIVOT ( FOR (course, s) IN (('dotNET', (1, 'a')), ('Java', (2, 'b'))) ) -- !query schema -struct +struct -- !query output 2012 15000 NULL 2013 NULL 30000 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index 9d9625c89c383..c0f847ba56648 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -1421,6 +1421,38 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { } } + test("withField user-facing examples") { + checkAnswer( + sql("SELECT named_struct('a', 1, 'b', 2) struct_col") + .select($"struct_col".withField("c", lit(3))), + Row(Row(1, 2, 3))) + + checkAnswer( + sql("SELECT named_struct('a', 1, 'b', 2) struct_col") + .select($"struct_col".withField("b", lit(3))), + Row(Row(1, 3))) + + checkAnswer( + sql("SELECT CAST(NULL AS struct) struct_col") + .select($"struct_col".withField("c", lit(3))), + Row(null)) + + checkAnswer( + sql("SELECT named_struct('a', 1, 'b', 2, 'b', 3) struct_col") + .select($"struct_col".withField("b", lit(100))), + Row(Row(1, 100, 100))) + + checkAnswer( + sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col") + .select($"struct_col".withField("a.c", lit(3))), + Row(Row(Row(1, 2, 3)))) + + intercept[AnalysisException] { + sql("SELECT named_struct('a', named_struct('b', 1), 'a', named_struct('c', 2)) struct_col") + .select($"struct_col".withField("a.c", lit(3))) + }.getMessage should include("Ambiguous reference to fields") + } + test("dropFields should throw an exception if called on a non-StructType column") { intercept[AnalysisException] { testData.withColumn("key", $"key".dropFields("a")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 0d0e91e2287e0..bcfc77545bbd2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -1268,7 +1268,7 @@ class DataFrameSuite extends QueryTest s"""+----------------+ || a| |+----------------+ - ||[1 -> a, 2 -> b]| + ||{1 -> a, 2 -> b}| |+----------------+ |""".stripMargin) val df3 = Seq(((1, "a"), 0), ((2, "b"), 0)).toDF("a", "b") @@ -1276,8 +1276,8 @@ class DataFrameSuite extends QueryTest s"""+------+---+ || a| b| |+------+---+ - ||[1, a]| 0| - ||[2, b]| 0| + ||{1, a}| 0| + ||{2, b}| 0| |+------+---+ |""".stripMargin) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 06600c1e4b1d7..4923e8b556907 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -1116,8 +1116,8 @@ class DatasetSuite extends QueryTest """+--------+ || f| |+--------+ - ||[foo, 1]| - ||[bar, 2]| + ||{foo, 1}| + ||{bar, 2}| |+--------+ |""".stripMargin diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala index 0b754e9e3ec0b..8edfb91d15fff 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -1240,6 +1240,30 @@ abstract class DynamicPartitionPruningSuiteBase } } + test("SPARK-32509: Unused Dynamic Pruning filter shouldn't affect " + + "canonicalization and exchange reuse") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val df = sql( + """ WITH view1 as ( + | SELECT f.store_id FROM fact_stats f WHERE f.units_sold = 70 + | ) + | + | SELECT * FROM view1 v1 join view1 v2 WHERE v1.store_id = v2.store_id + """.stripMargin) + + checkPartitionPruningPredicate(df, false, false) + val reuseExchangeNodes = df.queryExecution.executedPlan.collect { + case se: ReusedExchangeExec => se + } + assert(reuseExchangeNodes.size == 1, "Expected plan to contain 1 ReusedExchangeExec " + + s"nodes. Found ${reuseExchangeNodes.size}") + + checkAnswer(df, Row(15, 15) :: Nil) + } + } + } + test("Plan broadcast pruning only when the broadcast can be reused") { Given("dynamic pruning filter on the build side") withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/NestedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/NestedDataSourceSuite.scala index 152d59b7b1908..78b314272aa07 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/NestedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/NestedDataSourceSuite.scala @@ -24,6 +24,16 @@ import org.apache.spark.sql.types.{LongType, StructType} // Datasource tests for nested schemas trait NestedDataSourceSuiteBase extends QueryTest with SharedSparkSession { protected val nestedDataSources: Seq[String] = Seq("orc", "parquet", "json") + protected def readOptions(schema: StructType): Map[String, String] = Map.empty + protected def save(selectExpr: Seq[String], format: String, path: String): Unit = { + spark + .range(1L) + .selectExpr(selectExpr: _*) + .write.mode("overwrite") + .format(format) + .save(path) + } + protected val colType: String = "in the data schema" test("SPARK-32431: consistent error for nested and top-level duplicate columns") { Seq( @@ -44,22 +54,17 @@ trait NestedDataSourceSuiteBase extends QueryTest with SharedSparkSession { withClue(s"format = $format select = ${selectExpr.mkString(",")}") { withTempPath { dir => val path = dir.getCanonicalPath - spark - .range(1L) - .selectExpr(selectExpr: _*) - .write.mode("overwrite") - .format(format) - .save(path) + save(selectExpr, format, path) val e = intercept[AnalysisException] { spark .read + .options(readOptions(caseInsensitiveSchema)) .schema(caseInsensitiveSchema) .format(format) .load(path) .show } - assert(e.getMessage.contains( - "Found duplicate column(s) in the data schema: `camelcase`")) + assert(e.getMessage.contains(s"Found duplicate column(s) $colType: `camelcase`")) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala index cc261a9ed3598..9da32d02aa723 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql import org.scalatest.BeforeAndAfterEach import org.apache.spark.{SparkConf, SparkContext, SparkException, SparkFunSuite} -import org.apache.spark.internal.config.ALLOW_SPARK_CONTEXT_IN_EXECUTORS +import org.apache.spark.internal.config.EXECUTOR_ALLOW_SPARK_CONTEXT import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf._ @@ -277,7 +277,7 @@ class SparkSessionBuilderSuite extends SparkFunSuite with BeforeAndAfterEach { session.range(1).foreach { v => SparkSession.builder.master("local") - .config(ALLOW_SPARK_CONTEXT_IN_EXECUTORS.key, true).getOrCreate().stop() + .config(EXECUTOR_ALLOW_SPARK_CONTEXT.key, true).getOrCreate().stop() () } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala index f904b53fe47eb..8d5166b5398cc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala @@ -154,8 +154,25 @@ class StringFunctionsSuite extends QueryTest with SharedSparkSession { Row("300", "100") :: Row("400", "100") :: Row("400-400", "100") :: Nil) } + test("string regex_extract_all") { + val df = Seq( + ("100-200,300-400", "(\\d+)-(\\d+)"), + ("101-201,301-401", "(\\d+)-(\\d+)"), + ("102-202,302-402", "(\\d+)")).toDF("a", "b") + + checkAnswer( + df.select( + regexp_extract_all($"a", "(\\d+)-(\\d+)", 1), + regexp_extract_all($"a", "(\\d+)-(\\d+)", 2)), + Row(Seq("100", "300"), Seq("200", "400")) :: + Row(Seq("101", "301"), Seq("201", "401")) :: + Row(Seq("102", "302"), Seq("202", "402")) :: Nil) + } + test("non-matching optional group") { val df = Seq(Tuple1("aaaac")).toDF("s") + + // regexp_extract checkAnswer( df.select(regexp_extract($"s", "(foo)", 1)), Row("") @@ -164,6 +181,16 @@ class StringFunctionsSuite extends QueryTest with SharedSparkSession { df.select(regexp_extract($"s", "(a+)(b)?(c)", 2)), Row("") ) + + // regexp_extract_all + checkAnswer( + df.select(regexp_extract_all($"s", "(foo)", 1)), + Row(Seq()) + ) + checkAnswer( + df.select(regexp_extract_all($"s", "(a+)(b)?(c)", 2)), + Row(Seq("")) + ) } test("string ascii function") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index db85ae613eaa1..04af7d1a68682 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -550,7 +550,7 @@ class UDFSuite extends QueryTest with SharedSparkSession { udf((d: LocalDate, i: Instant) => LocalDateInstantType(d, i))) checkAnswer(df.selectExpr(s"buildLocalDateInstantType(d, i) as di") .select('di.cast(StringType)), - Row(s"[$expectedDate, $expectedInstant]") :: Nil) + Row(s"{$expectedDate, $expectedInstant}") :: Nil) // test null cases spark.udf.register("buildLocalDateInstantType", @@ -580,7 +580,7 @@ class UDFSuite extends QueryTest with SharedSparkSession { udf((t: Timestamp, i: Instant) => TimestampInstantType(t, i))) checkAnswer(df.selectExpr("buildTimestampInstantType(t, i) as ti") .select('ti.cast(StringType)), - Row(s"[$expectedTimestamp, $expectedInstant]")) + Row(s"{$expectedTimestamp, $expectedInstant}")) // test null cases spark.udf.register("buildTimestampInstantType", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala index 1991f139e48c5..62712cf72eb59 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -17,6 +17,9 @@ package org.apache.spark.sql.execution +import scala.collection.JavaConverters._ + +import org.apache.spark.internal.config.ConfigEntry import org.apache.spark.sql.SaveMode import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedAlias, UnresolvedAttribute, UnresolvedRelation, UnresolvedStar} @@ -25,7 +28,7 @@ import org.apache.spark.sql.catalyst.expressions.{Ascending, Concat, SortOrder} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, RepartitionByExpression, Sort} import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.{CreateTable, RefreshResource} -import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} +import org.apache.spark.sql.internal.{HiveSerDe, SQLConf, StaticSQLConf} import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType} /** @@ -61,6 +64,82 @@ class SparkSqlParserSuite extends AnalysisTest { private def intercept(sqlCommand: String, messages: String*): Unit = interceptParseException(parser.parsePlan)(sqlCommand, messages: _*) + test("Checks if SET/RESET can parse all the configurations") { + // Force to build static SQL configurations + StaticSQLConf + ConfigEntry.knownConfigs.values.asScala.foreach { config => + assertEqual(s"SET ${config.key}", SetCommand(Some(config.key -> None))) + if (config.defaultValue.isDefined && config.defaultValueString != null) { + assertEqual(s"SET ${config.key}=${config.defaultValueString}", + SetCommand(Some(config.key -> Some(config.defaultValueString)))) + } + assertEqual(s"RESET ${config.key}", ResetCommand(Some(config.key))) + } + } + + test("Report Error for invalid usage of SET command") { + assertEqual("SET", SetCommand(None)) + assertEqual("SET -v", SetCommand(Some("-v", None))) + assertEqual("SET spark.sql.key", SetCommand(Some("spark.sql.key" -> None))) + assertEqual("SET spark.sql.key ", SetCommand(Some("spark.sql.key" -> None))) + assertEqual("SET spark:sql:key=false", SetCommand(Some("spark:sql:key" -> Some("false")))) + assertEqual("SET spark:sql:key=", SetCommand(Some("spark:sql:key" -> Some("")))) + assertEqual("SET spark:sql:key= ", SetCommand(Some("spark:sql:key" -> Some("")))) + assertEqual("SET spark:sql:key=-1 ", SetCommand(Some("spark:sql:key" -> Some("-1")))) + assertEqual("SET spark:sql:key = -1", SetCommand(Some("spark:sql:key" -> Some("-1")))) + assertEqual("SET 1.2.key=value", SetCommand(Some("1.2.key" -> Some("value")))) + assertEqual("SET spark.sql.3=4", SetCommand(Some("spark.sql.3" -> Some("4")))) + assertEqual("SET 1:2:key=value", SetCommand(Some("1:2:key" -> Some("value")))) + assertEqual("SET spark:sql:3=4", SetCommand(Some("spark:sql:3" -> Some("4")))) + assertEqual("SET 5=6", SetCommand(Some("5" -> Some("6")))) + assertEqual("SET spark:sql:key = va l u e ", + SetCommand(Some("spark:sql:key" -> Some("va l u e")))) + assertEqual("SET `spark.sql. key`=value", + SetCommand(Some("spark.sql. key" -> Some("value")))) + assertEqual("SET `spark.sql. key`= v a lu e ", + SetCommand(Some("spark.sql. key" -> Some("v a lu e")))) + assertEqual("SET `spark.sql. key`= -1", + SetCommand(Some("spark.sql. key" -> Some("-1")))) + + val expectedErrMsg = "Expected format is 'SET', 'SET key', or " + + "'SET key=value'. If you want to include special characters in key, " + + "please use quotes, e.g., SET `ke y`=value." + intercept("SET spark.sql.key value", expectedErrMsg) + intercept("SET spark.sql.key 'value'", expectedErrMsg) + intercept("SET spark.sql.key \"value\" ", expectedErrMsg) + intercept("SET spark.sql.key value1 value2", expectedErrMsg) + intercept("SET spark. sql.key=value", expectedErrMsg) + intercept("SET spark :sql:key=value", expectedErrMsg) + intercept("SET spark . sql.key=value", expectedErrMsg) + intercept("SET spark.sql. key=value", expectedErrMsg) + intercept("SET spark.sql :key=value", expectedErrMsg) + intercept("SET spark.sql . key=value", expectedErrMsg) + } + + test("Report Error for invalid usage of RESET command") { + assertEqual("RESET", ResetCommand(None)) + assertEqual("RESET spark.sql.key", ResetCommand(Some("spark.sql.key"))) + assertEqual("RESET spark.sql.key ", ResetCommand(Some("spark.sql.key"))) + assertEqual("RESET 1.2.key ", ResetCommand(Some("1.2.key"))) + assertEqual("RESET spark.sql.3", ResetCommand(Some("spark.sql.3"))) + assertEqual("RESET 1:2:key ", ResetCommand(Some("1:2:key"))) + assertEqual("RESET spark:sql:3", ResetCommand(Some("spark:sql:3"))) + assertEqual("RESET `spark.sql. key`", ResetCommand(Some("spark.sql. key"))) + + val expectedErrMsg = "Expected format is 'RESET' or 'RESET key'. " + + "If you want to include special characters in key, " + + "please use quotes, e.g., RESET `ke y`." + intercept("RESET spark.sql.key1 key2", expectedErrMsg) + intercept("RESET spark. sql.key1 key2", expectedErrMsg) + intercept("RESET spark.sql.key1 key2 key3", expectedErrMsg) + intercept("RESET spark: sql:key", expectedErrMsg) + intercept("RESET spark .sql.key", expectedErrMsg) + intercept("RESET spark : sql:key", expectedErrMsg) + intercept("RESET spark.sql: key", expectedErrMsg) + intercept("RESET spark.sql .key", expectedErrMsg) + intercept("RESET spark.sql : key", expectedErrMsg) + } + test("refresh resource") { assertEqual("REFRESH prefix_path", RefreshResource("prefix_path")) assertEqual("REFRESH /", RefreshResource("/")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/CachedBatchSerializerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/CachedBatchSerializerSuite.scala new file mode 100644 index 0000000000000..099a1aa996c11 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/CachedBatchSerializerSuite.scala @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.columnar + +import org.apache.spark.SparkConf +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{QueryTest, Row} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.columnar.{CachedBatch, CachedBatchSerializer} +import org.apache.spark.sql.execution.columnar.InMemoryRelation.clearSerializer +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} +import org.apache.spark.storage.StorageLevel + +case class SingleIntCachedBatch(data: Array[Int]) extends CachedBatch { + override def numRows: Int = data.length + override def sizeInBytes: Long = 4 * data.length +} + +/** + * Very simple serializer that only supports a single int column, but does support columnar. + */ +class TestSingleIntColumnarCachedBatchSerializer extends CachedBatchSerializer { + override def supportsColumnarInput(schema: Seq[Attribute]): Boolean = true + + override def convertInternalRowToCachedBatch( + input: RDD[InternalRow], + schema: Seq[Attribute], + storageLevel: StorageLevel, + conf: SQLConf): RDD[CachedBatch] = { + throw new IllegalStateException("This does not work. This is only for testing") + } + + override def convertColumnarBatchToCachedBatch( + input: RDD[ColumnarBatch], + schema: Seq[Attribute], + storageLevel: StorageLevel, + conf: SQLConf): RDD[CachedBatch] = { + if (schema.length != 1 || schema.head.dataType != IntegerType) { + throw new IllegalArgumentException("Only a single column of non-nullable ints works. " + + s"This is for testing $schema") + } + input.map { cb => + val column = cb.column(0) + val data = column.getInts(0, cb.numRows()) + SingleIntCachedBatch(data) + } + } + + override def supportsColumnarOutput(schema: StructType): Boolean = true + override def vectorTypes(attributes: Seq[Attribute], conf: SQLConf): Option[Seq[String]] = + Some(attributes.map(_ => classOf[OnHeapColumnVector].getName)) + + override def convertCachedBatchToColumnarBatch( + input: RDD[CachedBatch], + cacheAttributes: Seq[Attribute], + selectedAttributes: Seq[Attribute], + conf: SQLConf): RDD[ColumnarBatch] = { + if (selectedAttributes.isEmpty) { + input.map { cached => + val single = cached.asInstanceOf[SingleIntCachedBatch] + new ColumnarBatch(new Array[ColumnVector](0), single.numRows) + } + } else { + if (selectedAttributes.length > 1 || + selectedAttributes.head.dataType != IntegerType) { + throw new IllegalArgumentException("Only a single column of non-nullable ints works. " + + s"This is for testing") + } + input.map { cached => + val single = cached.asInstanceOf[SingleIntCachedBatch] + val cv = OnHeapColumnVector.allocateColumns(single.numRows, selectedAttributes.toStructType) + val data = single.data + cv(0).putInts(0, data.length, data, 0) + new ColumnarBatch(cv.toArray, single.numRows) + } + } + } + + override def convertCachedBatchToInternalRow( + input: RDD[CachedBatch], + cacheAttributes: Seq[Attribute], + selectedAttributes: Seq[Attribute], + conf: SQLConf): RDD[InternalRow] = { + throw new IllegalStateException("This does not work. This is only for testing") + } + + override def buildFilter( + predicates: Seq[Expression], + cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] = { + def ret(index: Int, cb: Iterator[CachedBatch]): Iterator[CachedBatch] = cb + ret + } +} + +class CachedBatchSerializerSuite extends QueryTest with SharedSparkSession { + import testImplicits._ + + override protected def sparkConf: SparkConf = { + super.sparkConf.set( + StaticSQLConf.SPARK_CACHE_SERIALIZER.key, + classOf[TestSingleIntColumnarCachedBatchSerializer].getName) + } + + protected override def beforeAll(): Unit = { + super.beforeAll() + clearSerializer() + } + + protected override def afterAll(): Unit = { + clearSerializer() + super.afterAll() + } + + test("Columnar Cache Plugin") { + withTempPath { workDir => + val workDirPath = workDir.getAbsolutePath + val input = Seq(100, 200, 300).toDF("count") + input.write.parquet(workDirPath) + val data = spark.read.parquet(workDirPath) + data.cache() + assert(data.count() == 3) + checkAnswer(data, Row(100) :: Row(200) :: Row(300) :: Nil) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala index 18f29f7b90ad5..b8f73f4563ef1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala @@ -20,18 +20,32 @@ package org.apache.spark.sql.execution.columnar import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} +import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, QueryTest, Row} -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, In} -import org.apache.spark.sql.catalyst.plans.logical.LocalRelation +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, AttributeSet, In} import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning -import org.apache.spark.sql.execution.{ColumnarToRowExec, FilterExec, InputAdapter, LocalTableScanExec, WholeStageCodegenExec} +import org.apache.spark.sql.columnar.CachedBatch +import org.apache.spark.sql.execution.{ColumnarToRowExec, FilterExec, InputAdapter, WholeStageCodegenExec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.test.SQLTestData._ import org.apache.spark.sql.types._ +import org.apache.spark.storage.StorageLevel import org.apache.spark.storage.StorageLevel._ -import org.apache.spark.util.Utils + +class TestCachedBatchSerializer( + useCompression: Boolean, + batchSize: Int) extends DefaultCachedBatchSerializer { + + override def convertInternalRowToCachedBatch(input: RDD[InternalRow], + schema: Seq[Attribute], + storageLevel: StorageLevel, + conf: SQLConf): RDD[CachedBatch] = { + convertForCacheInternal(input, schema, batchSize, useCompression) + } +} class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { import testImplicits._ @@ -42,12 +56,12 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { data.createOrReplaceTempView(s"testData$dataType") val storageLevel = MEMORY_ONLY val plan = spark.sessionState.executePlan(data.logicalPlan).sparkPlan - val inMemoryRelation = InMemoryRelation(useCompression = true, 5, storageLevel, plan, None, - data.logicalPlan) + val inMemoryRelation = InMemoryRelation(new TestCachedBatchSerializer(useCompression = true, 5), + storageLevel, plan, None, data.logicalPlan) assert(inMemoryRelation.cacheBuilder.cachedColumnBuffers.getStorageLevel == storageLevel) inMemoryRelation.cacheBuilder.cachedColumnBuffers.collect().head match { - case _: CachedBatch => + case _: DefaultCachedBatch => case other => fail(s"Unexpected cached batch type: ${other.getClass.getName}") } checkAnswer(inMemoryRelation, data.collect().toSeq) @@ -119,8 +133,8 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { test("simple columnar query") { val plan = spark.sessionState.executePlan(testData.logicalPlan).sparkPlan - val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan, None, - testData.logicalPlan) + val scan = InMemoryRelation(new TestCachedBatchSerializer(useCompression = true, 5), + MEMORY_ONLY, plan, None, testData.logicalPlan) checkAnswer(scan, testData.collect().toSeq) } @@ -140,8 +154,8 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { test("projection") { val logicalPlan = testData.select('value, 'key).logicalPlan val plan = spark.sessionState.executePlan(logicalPlan).sparkPlan - val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan, None, - logicalPlan) + val scan = InMemoryRelation(new TestCachedBatchSerializer(useCompression = true, 5), + MEMORY_ONLY, plan, None, logicalPlan) checkAnswer(scan, testData.collect().map { case Row(key: Int, value: String) => value -> key @@ -157,8 +171,8 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { test("SPARK-1436 regression: in-memory columns must be able to be accessed multiple times") { val plan = spark.sessionState.executePlan(testData.logicalPlan).sparkPlan - val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan, None, - testData.logicalPlan) + val scan = InMemoryRelation(new TestCachedBatchSerializer(useCompression = true, 5), + MEMORY_ONLY, plan, None, testData.logicalPlan) checkAnswer(scan, testData.collect().toSeq) checkAnswer(scan, testData.collect().toSeq) @@ -336,7 +350,8 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { test("SPARK-17549: cached table size should be correctly calculated") { val data = spark.sparkContext.parallelize(1 to 10, 5).toDF() val plan = spark.sessionState.executePlan(data.logicalPlan).sparkPlan - val cached = InMemoryRelation(true, 5, MEMORY_ONLY, plan, None, data.logicalPlan) + val cached = InMemoryRelation(new TestCachedBatchSerializer(true, 5), + MEMORY_ONLY, plan, None, data.logicalPlan) // Materialize the data. val expectedAnswer = data.collect() @@ -349,7 +364,8 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { test("cached row count should be calculated") { val data = spark.range(6).toDF val plan = spark.sessionState.executePlan(data.logicalPlan).sparkPlan - val cached = InMemoryRelation(true, 5, MEMORY_ONLY, plan, None, data.logicalPlan) + val cached = InMemoryRelation(new TestCachedBatchSerializer(true, 5), + MEMORY_ONLY, plan, None, data.logicalPlan) // Materialize the data. val expectedAnswer = data.collect() @@ -474,12 +490,8 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { test("SPARK-22249: buildFilter should not throw exception when In contains an empty list") { val attribute = AttributeReference("a", IntegerType)() - val localTableScanExec = LocalTableScanExec(Seq(attribute), Nil) - val testRelation = InMemoryRelation(false, 1, MEMORY_ONLY, localTableScanExec, None, - LocalRelation(Seq(attribute), Nil)) - val tableScanExec = InMemoryTableScanExec(Seq(attribute), - Seq(In(attribute, Nil)), testRelation) - assert(tableScanExec.partitionFilters.isEmpty) + val testSerializer = new TestCachedBatchSerializer(false, 1) + testSerializer.buildFilter(Seq(In(attribute, Nil)), Seq(attribute)) } testWithWholeStageCodegenOnAndOff("SPARK-22348: table cache " + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala index c399a011f9073..c3bcf86c1ed27 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala @@ -17,17 +17,19 @@ package org.apache.spark.sql.execution.datasources.v2 +import java.net.URI import java.util import java.util.Collections import scala.collection.JavaConverters._ +import org.apache.hadoop.fs.Path import org.scalatest.BeforeAndAfter import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser -import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, NamespaceChange, SupportsNamespaces, TableChange} +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, NamespaceChange, SupportsNamespaces, TableCatalog, TableChange, V1Table} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{DoubleType, IntegerType, LongType, StringType, StructField, StructType, TimestampType} import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -160,6 +162,36 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { assert(catalog.tableExists(testIdent)) } + private def makeQualifiedPathWithWarehouse(path: String): URI = { + val p = new Path(spark.sessionState.conf.warehousePath, path) + val fs = p.getFileSystem(spark.sessionState.newHadoopConf()) + fs.makeQualified(p).toUri + + } + + test("createTable: location") { + val catalog = newCatalog() + val properties = new util.HashMap[String, String]() + assert(!catalog.tableExists(testIdent)) + + // default location + val t1 = catalog.createTable(testIdent, schema, Array.empty, properties).asInstanceOf[V1Table] + assert(t1.catalogTable.location === + spark.sessionState.catalog.defaultTablePath(testIdent.asTableIdentifier)) + catalog.dropTable(testIdent) + + // relative path + properties.put(TableCatalog.PROP_LOCATION, "relative/path") + val t2 = catalog.createTable(testIdent, schema, Array.empty, properties).asInstanceOf[V1Table] + assert(t2.catalogTable.location === makeQualifiedPathWithWarehouse("db.db/relative/path")) + catalog.dropTable(testIdent) + + // absolute path + properties.put(TableCatalog.PROP_LOCATION, "/absolute/path") + val t3 = catalog.createTable(testIdent, schema, Array.empty, properties).asInstanceOf[V1Table] + assert(t3.catalogTable.location.toString === "file:/absolute/path") + } + test("tableExists") { val catalog = newCatalog() @@ -640,6 +672,26 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { assert(exc.message.contains("not found")) } + test("alterTable: location") { + val catalog = newCatalog() + assert(!catalog.tableExists(testIdent)) + + // default location + val t1 = catalog.createTable(testIdent, schema, Array.empty, emptyProps).asInstanceOf[V1Table] + assert(t1.catalogTable.location === + spark.sessionState.catalog.defaultTablePath(testIdent.asTableIdentifier)) + + // relative path + val t2 = catalog.alterTable(testIdent, + TableChange.setProperty(TableCatalog.PROP_LOCATION, "relative/path")).asInstanceOf[V1Table] + assert(t2.catalogTable.location === makeQualifiedPathWithWarehouse("db.db/relative/path")) + + // absolute path + val t3 = catalog.alterTable(testIdent, + TableChange.setProperty(TableCatalog.PROP_LOCATION, "/absolute/path")).asInstanceOf[V1Table] + assert(t3.catalogTable.location.toString === "file:/absolute/path") + } + test("dropTable") { val catalog = newCatalog() @@ -812,11 +864,15 @@ class V2SessionCatalogNamespaceSuite extends V2SessionCatalogBaseSuite { test("createNamespace: basic behavior") { val catalog = newCatalog() - val expectedPath = sqlContext.sessionState.catalog.getDefaultDBPath(testNs(0)).toString + + val sessionCatalog = sqlContext.sessionState.catalog + val expectedPath = + new Path(spark.sessionState.conf.warehousePath, + sessionCatalog.getDefaultDBPath(testNs(0)).toString).toString catalog.createNamespace(testNs, Map("property" -> "value").asJava) - assert(expectedPath === spark.catalog.getDatabase(testNs(0)).locationUri.toString) + assert(expectedPath === spark.catalog.getDatabase(testNs(0)).locationUri) assert(catalog.namespaceExists(testNs) === true) val metadata = catalog.loadNamespaceMetadata(testNs).asScala @@ -842,6 +898,23 @@ class V2SessionCatalogNamespaceSuite extends V2SessionCatalogBaseSuite { catalog.dropNamespace(testNs) } + test("createNamespace: relative location") { + val catalog = newCatalog() + val expectedPath = + new Path(spark.sessionState.conf.warehousePath, "a/b/c").toString + + catalog.createNamespace(testNs, Map("location" -> "a/b/c").asJava) + + assert(expectedPath === spark.catalog.getDatabase(testNs(0)).locationUri) + + assert(catalog.namespaceExists(testNs) === true) + val metadata = catalog.loadNamespaceMetadata(testNs).asScala + checkMetadata(metadata, Map.empty) + assert(expectedPath === metadata("location")) + + catalog.dropNamespace(testNs) + } + test("createNamespace: fail if namespace already exists") { val catalog = newCatalog() @@ -954,16 +1027,23 @@ class V2SessionCatalogNamespaceSuite extends V2SessionCatalogBaseSuite { test("alterNamespace: update namespace location") { val catalog = newCatalog() - val initialPath = sqlContext.sessionState.catalog.getDefaultDBPath(testNs(0)).toString - val newPath = "file:/tmp/db.db" + val initialPath = + new Path(spark.sessionState.conf.warehousePath, + spark.sessionState.catalog.getDefaultDBPath(testNs(0)).toString).toString + val newAbsoluteUri = "file:/tmp/db.db" catalog.createNamespace(testNs, emptyProps) + assert(initialPath === spark.catalog.getDatabase(testNs(0)).locationUri) + catalog.alterNamespace(testNs, NamespaceChange.setProperty("location", newAbsoluteUri)) + assert(newAbsoluteUri === spark.catalog.getDatabase(testNs(0)).locationUri) - assert(initialPath === spark.catalog.getDatabase(testNs(0)).locationUri.toString) - - catalog.alterNamespace(testNs, NamespaceChange.setProperty("location", newPath)) + val newAbsolutePath = "/tmp/newAbsolutePath" + catalog.alterNamespace(testNs, NamespaceChange.setProperty("location", newAbsolutePath)) + assert("file:" + newAbsolutePath === spark.catalog.getDatabase(testNs(0)).locationUri) - assert(newPath === spark.catalog.getDatabase(testNs(0)).locationUri.toString) + val newRelativePath = new Path(spark.sessionState.conf.warehousePath, "relativeP").toString + catalog.alterNamespace(testNs, NamespaceChange.setProperty("location", "relativeP")) + assert(newRelativePath === spark.catalog.getDatabase(testNs(0)).locationUri) catalog.dropNamespace(testNs) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 565a8351d9c06..5701cc917258b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -115,7 +115,7 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { sql(s"set ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS}=10") assert(spark.conf.get(SQLConf.SHUFFLE_PARTITIONS) === 10) } finally { - sql(s"set ${SQLConf.SHUFFLE_PARTITIONS}=$original") + sql(s"set ${SQLConf.SHUFFLE_PARTITIONS.key}=$original") } } @@ -149,7 +149,7 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { assert(spark.conf.get(SQLConf.OPTIMIZER_EXCLUDED_RULES) === Some("org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation")) } finally { - sql(s"set ${SQLConf.GROUP_BY_ORDINAL}=$original") + sql(s"set ${SQLConf.GROUP_BY_ORDINAL.key}=$original") } } @@ -165,7 +165,7 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { assert(spark.conf.get(SQLConf.OPTIMIZER_MAX_ITERATIONS) === 100) assert(sql(s"set").where(s"key = '${SQLConf.OPTIMIZER_MAX_ITERATIONS.key}'").count() == 0) } finally { - sql(s"set ${SQLConf.OPTIMIZER_MAX_ITERATIONS}=$original") + sql(s"set ${SQLConf.OPTIMIZER_MAX_ITERATIONS.key}=$original") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCNestedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCNestedDataSourceSuite.scala new file mode 100644 index 0000000000000..46bdb1918147c --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCNestedDataSourceSuite.scala @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.jdbc + +import org.apache.spark.sql.NestedDataSourceSuiteBase +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.Utils + +class JDBCNestedDataSourceSuite extends NestedDataSourceSuiteBase { + override val nestedDataSources: Seq[String] = Seq("jdbc") + private val tempDir = Utils.createTempDir() + private val url = s"jdbc:h2:${tempDir.getCanonicalPath};user=testUser;password=testPass" + override val colType: String = "in the customSchema option value" + + override def afterAll(): Unit = { + Utils.deleteRecursively(tempDir) + super.afterAll() + } + + override def readOptions(schema: StructType): Map[String, String] = { + Map("url" -> url, "dbtable" -> "t1", "customSchema" -> schema.toDDL) + } + + override def save(selectExpr: Seq[String], format: String, path: String): Unit = { + // We ignore `selectExpr` because: + // 1. H2 doesn't support nested columns + // 2. JDBC datasource checks duplicates before comparing of user's schema with + // actual schema of `t1`. + spark + .range(1L) + .write.mode("overwrite") + .options(Map("url" -> url, "dbtable" -> "t1")) + .format(format) + .save() + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala index d42732f426681..069517acd68cc 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala @@ -17,12 +17,10 @@ package org.apache.spark.sql.hive.thriftserver -import java.util.UUID import java.util.regex.Pattern import scala.collection.JavaConverters.seqAsJavaListConverter -import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.hive.ql.security.authorization.plugin.{HiveOperationType, HivePrivilegeObject} import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType import org.apache.hive.service.cli._ @@ -34,7 +32,7 @@ import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.SessionCatalog import org.apache.spark.sql.hive.thriftserver.ThriftserverShimUtils.toJavaSQLType -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types._ /** * Spark's own SparkGetColumnsOperation @@ -126,12 +124,52 @@ private[hive] class SparkGetColumnsOperation( HiveThriftServer2.eventManager.onStatementFinish(statementId) } + /** + * For boolean, numeric and datetime types, it returns the default size of its catalyst type + * For struct type, when its elements are fixed-size, the summation of all element sizes will be + * returned. + * For array, map, string, and binaries, the column size is variable, return null as unknown. + */ + private def getColumnSize(typ: DataType): Option[Int] = typ match { + case dt @ (BooleanType | _: NumericType | DateType | TimestampType) => Some(dt.defaultSize) + case StructType(fields) => + val sizeArr = fields.map(f => getColumnSize(f.dataType)) + if (sizeArr.contains(None)) { + None + } else { + Some(sizeArr.map(_.get).sum) + } + case other => None + } + + /** + * The number of fractional digits for this type. + * Null is returned for data types where this is not applicable. + * For boolean and integrals, the decimal digits is 0 + * For floating types, we follow the IEEE Standard for Floating-Point Arithmetic (IEEE 754) + * For timestamp values, we support microseconds + * For decimals, it returns the scale + */ + private def getDecimalDigits(typ: DataType) = typ match { + case BooleanType | _: IntegerType => Some(0) + case FloatType => Some(7) + case DoubleType => Some(15) + case d: DecimalType => Some(d.scale) + case TimestampType => Some(6) + case _ => None + } + + private def getNumPrecRadix(typ: DataType): Option[Int] = typ match { + case _: NumericType => Some(10) + case _ => None + } + private def addToRowSet( columnPattern: Pattern, dbName: String, tableName: String, schema: StructType): Unit = { - schema.foreach { column => + schema.zipWithIndex.foreach { case (column, pos) => if (columnPattern != null && !columnPattern.matcher(column.name).matches()) { } else { val rowData = Array[AnyRef]( @@ -141,17 +179,17 @@ private[hive] class SparkGetColumnsOperation( column.name, // COLUMN_NAME toJavaSQLType(column.dataType.sql).asInstanceOf[AnyRef], // DATA_TYPE column.dataType.sql, // TYPE_NAME - null, // COLUMN_SIZE + getColumnSize(column.dataType).map(_.asInstanceOf[AnyRef]).orNull, // COLUMN_SIZE null, // BUFFER_LENGTH, unused - null, // DECIMAL_DIGITS - null, // NUM_PREC_RADIX + getDecimalDigits(column.dataType).map(_.asInstanceOf[AnyRef]).orNull, // DECIMAL_DIGITS + getNumPrecRadix(column.dataType).map(_.asInstanceOf[AnyRef]).orNull, // NUM_PREC_RADIX (if (column.nullable) 1 else 0).asInstanceOf[AnyRef], // NULLABLE column.getComment().getOrElse(""), // REMARKS null, // COLUMN_DEF null, // SQL_DATA_TYPE null, // SQL_DATETIME_SUB null, // CHAR_OCTET_LENGTH - null, // ORDINAL_POSITION + pos.asInstanceOf[AnyRef], // ORDINAL_POSITION "YES", // IS_NULLABLE null, // SCOPE_CATALOG null, // SCOPE_SCHEMA diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala index 2bb9169693a15..196c093aafddf 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala @@ -19,9 +19,12 @@ package org.apache.spark.sql.hive.thriftserver import java.sql.SQLException +import scala.collection.JavaConverters._ + import org.apache.hive.service.cli.HiveSQLException import org.apache.spark.sql.hive.HiveUtils +import org.apache.spark.sql.types._ trait ThriftServerWithSparkContextSuite extends SharedThriftServer { @@ -101,6 +104,135 @@ trait ThriftServerWithSparkContextSuite extends SharedThriftServer { } } } + + test("check results from get columns operation from thrift server") { + val schemaName = "default" + val tableName = "spark_get_col_operation" + val schema = new StructType() + .add("c0", "boolean", nullable = false, "0") + .add("c1", "tinyint", nullable = true, "1") + .add("c2", "smallint", nullable = false, "2") + .add("c3", "int", nullable = true, "3") + .add("c4", "long", nullable = false, "4") + .add("c5", "float", nullable = true, "5") + .add("c6", "double", nullable = false, "6") + .add("c7", "decimal(38, 20)", nullable = true, "7") + .add("c8", "decimal(10, 2)", nullable = false, "8") + .add("c9", "string", nullable = true, "9") + .add("c10", "array", nullable = false, "10") + .add("c11", "array", nullable = true, "11") + .add("c12", "map", nullable = false, "12") + .add("c13", "date", nullable = true, "13") + .add("c14", "timestamp", nullable = false, "14") + .add("c15", "struct", nullable = true, "15") + .add("c16", "binary", nullable = false, "16") + + val ddl = + s""" + |CREATE TABLE $schemaName.$tableName ( + | ${schema.toDDL} + |) + |using parquet""".stripMargin + + withCLIServiceClient { client => + val sessionHandle = client.openSession(user, "") + val confOverlay = new java.util.HashMap[java.lang.String, java.lang.String] + val opHandle = client.executeStatement(sessionHandle, ddl, confOverlay) + var status = client.getOperationStatus(opHandle) + while (!status.getState.isTerminal) { + Thread.sleep(10) + status = client.getOperationStatus(opHandle) + } + val getCol = client.getColumns(sessionHandle, "", schemaName, tableName, null) + val rowSet = client.fetchResults(getCol) + val columns = rowSet.toTRowSet.getColumns + + val catalogs = columns.get(0).getStringVal.getValues.asScala + assert(catalogs.forall(_.isEmpty), "catalog name mismatches") + + val schemas = columns.get(1).getStringVal.getValues.asScala + assert(schemas.forall(_ == schemaName), "schema name mismatches") + + val tableNames = columns.get(2).getStringVal.getValues.asScala + assert(tableNames.forall(_ == tableName), "table name mismatches") + + val columnNames = columns.get(3).getStringVal.getValues.asScala + columnNames.zipWithIndex.foreach { + case (v, i) => assert(v === "c" + i, "column name mismatches") + } + + val javaTypes = columns.get(4).getI32Val.getValues + import java.sql.Types._ + assert(javaTypes.get(0).intValue() === BOOLEAN) + assert(javaTypes.get(1).intValue() === TINYINT) + assert(javaTypes.get(2).intValue() === SMALLINT) + assert(javaTypes.get(3).intValue() === INTEGER) + assert(javaTypes.get(4).intValue() === BIGINT) + assert(javaTypes.get(5).intValue() === FLOAT) + assert(javaTypes.get(6).intValue() === DOUBLE) + assert(javaTypes.get(7).intValue() === DECIMAL) + assert(javaTypes.get(8).intValue() === DECIMAL) + assert(javaTypes.get(9).intValue() === VARCHAR) + assert(javaTypes.get(10).intValue() === ARRAY) + assert(javaTypes.get(11).intValue() === ARRAY) + assert(javaTypes.get(12).intValue() === JAVA_OBJECT) + assert(javaTypes.get(13).intValue() === DATE) + assert(javaTypes.get(14).intValue() === TIMESTAMP) + assert(javaTypes.get(15).intValue() === STRUCT) + assert(javaTypes.get(16).intValue() === BINARY) + + val typeNames = columns.get(5).getStringVal.getValues.asScala + typeNames.zip(schema).foreach { case (tName, f) => + assert(tName === f.dataType.sql) + } + + val colSize = columns.get(6).getI32Val.getValues.asScala + + colSize.zip(schema).foreach { case (size, f) => + f.dataType match { + case StringType | BinaryType | _: ArrayType | _: MapType => assert(size === 0) + case o => assert(size === o.defaultSize) + } + } + + val decimalDigits = columns.get(8).getI32Val.getValues.asScala + decimalDigits.zip(schema).foreach { case (dd, f) => + f.dataType match { + case BooleanType | _: IntegerType => assert(dd === 0) + case d: DecimalType => assert(dd === d.scale) + case FloatType => assert(dd === 7) + case DoubleType => assert(dd === 15) + case TimestampType => assert(dd === 6) + case _ => assert(dd === 0) // nulls + } + } + + val radixes = columns.get(9).getI32Val.getValues.asScala + radixes.zip(schema).foreach { case (radix, f) => + f.dataType match { + case _: NumericType => assert(radix === 10) + case _ => assert(radix === 0) // nulls + } + } + + val nullables = columns.get(10).getI32Val.getValues.asScala + assert(nullables.forall(_ === 1)) + + val comments = columns.get(11).getStringVal.getValues.asScala + comments.zip(schema).foreach { case (c, f) => assert(c === f.getComment().get) } + + val positions = columns.get(16).getI32Val.getValues.asScala + positions.zipWithIndex.foreach { case (pos, idx) => + assert(pos === idx, "the client columns disorder") + } + + val isNullables = columns.get(17).getStringVal.getValues.asScala + assert(isNullables.forall(_ === "YES")) + + val autoIncs = columns.get(22).getStringVal.getValues.asScala + assert(autoIncs.forall(_ === "NO")) + } + } }