diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html b/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html index b236857cb4..31ef04552b 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html @@ -92,6 +92,7 @@ limitations under the License. Off Heap Storage Memory Disk Used Cores + Resources Active Tasks Failed Tasks Complete Tasks diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js index 17cb68c302..11d7c77d0c 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js @@ -39,6 +39,19 @@ function formatStatus(status, type, row) { return "Dead" } +function formatResourceCells(resources) { + var result = "" + var count = 0 + $.each(resources, function (name, resInfo) { + if (count > 0) { + result += ", " + } + result += name + ': [' + resInfo.addresses.join(", ") + ']' + count += 1 + }); + return result +} + jQuery.extend(jQuery.fn.dataTableExt.oSort, { "title-numeric-pre": function (a) { var x = a.match(/title="*(-?[0-9\.]+)/)[1]; @@ -106,7 +119,7 @@ function totalDurationColor(totalGCTime, totalDuration) { } var sumOptionalColumns = [3, 4]; -var execOptionalColumns = [5, 6]; +var execOptionalColumns = [5, 6, 9]; var execDataTable; var sumDataTable; @@ -401,6 +414,7 @@ $(document).ready(function () { }, {data: 'diskUsed', render: formatBytes}, {data: 'totalCores'}, + {name: 'resourcesCol', data: 'resources', render: formatResourceCells, orderable: false}, { data: 'activeTasks', "fnCreatedCell": function (nTd, sData, oData, iRow, iCol) { @@ -446,7 +460,8 @@ $(document).ready(function () { "order": [[0, "asc"]], "columnDefs": [ {"visible": false, "targets": 5}, - {"visible": false, "targets": 6} + {"visible": false, "targets": 6}, + {"visible": false, "targets": 9} ] }; @@ -553,6 +568,7 @@ $(document).ready(function () { "
Select All
" + "
On Heap Memory
" + "
Off Heap Memory
" + + "
Resources
" + ""); reselectCheckboxesBasedOnTaskTableState(); @@ -584,8 +600,10 @@ $(document).ready(function () { var execCol = execDataTable.column(execColIdx); execCol.visible(!execCol.visible()); var sumColIdx = thisBox.attr("data-sum-col-idx"); - var sumCol = sumDataTable.column(sumColIdx); - sumCol.visible(!sumCol.visible()); + if (sumColIdx) { + var sumCol = sumDataTable.column(sumColIdx); + sumCol.visible(!sumCol.visible()); + } } }); diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala index 73b2dc26a8..741050027f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala @@ -72,7 +72,7 @@ private[spark] class HistoryAppStatusStore( source.totalGCTime, source.totalInputBytes, source.totalShuffleRead, source.totalShuffleWrite, source.isBlacklisted, source.maxMemory, source.addTime, source.removeTime, source.removeReason, newExecutorLogs, source.memoryMetrics, - source.blacklistedInStages, source.peakMemoryMetrics, source.attributes) + source.blacklistedInStages, source.peakMemoryMetrics, source.attributes, source.resources) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala index 95098445eb..a33b15354e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala @@ -26,12 +26,15 @@ import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.util.Utils private[spark] case class WorkerResourceInfo(name: String, addresses: Seq[String]) - extends ResourceAllocator(name, addresses) { + extends ResourceAllocator { + + override protected def resourceName = this.name + override protected def resourceAddresses = this.addresses def acquire(amount: Int): ResourceInformation = { val allocated = availableAddrs.take(amount) acquire(allocated) - new ResourceInformation(name, allocated.toArray) + new ResourceInformation(resourceName, allocated.toArray) } } diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala b/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala index 6f9b9c0e0b..e64fadc113 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala @@ -23,18 +23,20 @@ import org.apache.spark.SparkException import org.apache.spark.util.collection.OpenHashMap /** - * Class used to help executor/worker allocate resources - * Please note that this class is intended to be used in a single thread. - * @param name Resource name, e.g. gpu/fpga - * @param addresses Resource addresses provided by the executor/worker + * Trait used to help executor/worker allocate resources. + * Please note that this is intended to be used in a single thread. */ -class ResourceAllocator(name: String, addresses: Seq[String]) extends Serializable { +trait ResourceAllocator { + + protected def resourceName: String + protected def resourceAddresses: Seq[String] + /** * Map from an address to its availability, the value `true` means the address is available, * while value `false` means the address is assigned. * TODO Use [[OpenHashMap]] instead to gain better performance. */ - private val addressAvailabilityMap = mutable.HashMap(addresses.map(_ -> true): _*) + private lazy val addressAvailabilityMap = mutable.HashMap(resourceAddresses.map(_ -> true): _*) /** * Sequence of currently available resource addresses. @@ -59,15 +61,15 @@ class ResourceAllocator(name: String, addresses: Seq[String]) extends Serializab def acquire(addrs: Seq[String]): Unit = { addrs.foreach { address => if (!addressAvailabilityMap.contains(address)) { - throw new SparkException(s"Try to acquire an address that doesn't exist. $name address " + - s"$address doesn't exist.") + throw new SparkException(s"Try to acquire an address that doesn't exist. $resourceName " + + s"address $address doesn't exist.") } val isAvailable = addressAvailabilityMap(address) if (isAvailable) { addressAvailabilityMap(address) = false } else { - throw new SparkException(s"Try to acquire an address that is not available. $name " + - s"address $address is not available.") + throw new SparkException("Try to acquire an address that is not available. " + + s"$resourceName address $address is not available.") } } } @@ -80,14 +82,14 @@ class ResourceAllocator(name: String, addresses: Seq[String]) extends Serializab def release(addrs: Seq[String]): Unit = { addrs.foreach { address => if (!addressAvailabilityMap.contains(address)) { - throw new SparkException(s"Try to release an address that doesn't exist. $name address " + - s"$address doesn't exist.") + throw new SparkException(s"Try to release an address that doesn't exist. $resourceName " + + s"address $address doesn't exist.") } val isAvailable = addressAvailabilityMap(address) if (!isAvailable) { addressAvailabilityMap(address) = true } else { - throw new SparkException(s"Try to release an address that is not assigned. $name " + + throw new SparkException(s"Try to release an address that is not assigned. $resourceName " + s"address $address is not assigned.") } } diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceInformation.scala b/core/src/main/scala/org/apache/spark/resource/ResourceInformation.scala index 8b8bd7b45f..d5ac41b995 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceInformation.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceInformation.scala @@ -74,6 +74,16 @@ private[spark] object ResourceInformation { s"Here is a correct example: $exampleJson.", e) } } + + def parseJson(json: JValue): ResourceInformation = { + implicit val formats = DefaultFormats + try { + json.extract[ResourceInformationJson].toResourceInformation + } catch { + case NonFatal(e) => + throw new SparkException(s"Error parsing JSON into ResourceInformation:\n$json\n", e) + } + } } /** A case class to simplify JSON serialization of [[ResourceInformation]]. */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala index f05281e50b..02047609ed 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler -import org.apache.spark.resource.ResourceAllocator +import org.apache.spark.resource.{ResourceAllocator, ResourceInformation} /** * Class to hold information about a type of Resource on an Executor. This information is managed @@ -27,4 +27,8 @@ import org.apache.spark.resource.ResourceAllocator * @param addresses Resource addresses provided by the executor */ private[spark] class ExecutorResourceInfo(name: String, addresses: Seq[String]) - extends ResourceAllocator(name, addresses) + extends ResourceInformation(name, addresses.toArray) with ResourceAllocator { + + override protected def resourceName = this.name + override protected def resourceAddresses = this.addresses +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala index 435365d5b6..17907d88e5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala @@ -31,12 +31,12 @@ import org.apache.spark.scheduler.ExecutorResourceInfo * @param resourcesInfo The information of the currently available resources on the executor */ private[cluster] class ExecutorData( - val executorEndpoint: RpcEndpointRef, - val executorAddress: RpcAddress, - override val executorHost: String, - var freeCores: Int, - override val totalCores: Int, - override val logUrlMap: Map[String, String], - override val attributes: Map[String, String], - val resourcesInfo: Map[String, ExecutorResourceInfo] -) extends ExecutorInfo(executorHost, totalCores, logUrlMap, attributes) + val executorEndpoint: RpcEndpointRef, + val executorAddress: RpcAddress, + override val executorHost: String, + var freeCores: Int, + override val totalCores: Int, + override val logUrlMap: Map[String, String], + override val attributes: Map[String, String], + override val resourcesInfo: Map[String, ExecutorResourceInfo] +) extends ExecutorInfo(executorHost, totalCores, logUrlMap, attributes, resourcesInfo) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala index 3197e06fcd..5a4ad6e00e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala @@ -17,6 +17,7 @@ package org.apache.spark.scheduler.cluster import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.resource.ResourceInformation /** * :: DeveloperApi :: @@ -27,10 +28,19 @@ class ExecutorInfo( val executorHost: String, val totalCores: Int, val logUrlMap: Map[String, String], - val attributes: Map[String, String]) { + val attributes: Map[String, String], + val resourcesInfo: Map[String, ResourceInformation]) { def this(executorHost: String, totalCores: Int, logUrlMap: Map[String, String]) = { - this(executorHost, totalCores, logUrlMap, Map.empty) + this(executorHost, totalCores, logUrlMap, Map.empty, Map.empty) + } + + def this( + executorHost: String, + totalCores: Int, + logUrlMap: Map[String, String], + attributes: Map[String, String]) = { + this(executorHost, totalCores, logUrlMap, attributes, Map.empty) } def canEqual(other: Any): Boolean = other.isInstanceOf[ExecutorInfo] @@ -41,12 +51,13 @@ class ExecutorInfo( executorHost == that.executorHost && totalCores == that.totalCores && logUrlMap == that.logUrlMap && - attributes == that.attributes + attributes == that.attributes && + resourcesInfo == that.resourcesInfo case _ => false } override def hashCode(): Int = { - val state = Seq(executorHost, totalCores, logUrlMap, attributes) + val state = Seq(executorHost, totalCores, logUrlMap, attributes, resourcesInfo) state.map(_.hashCode()).foldLeft(0)((a, b) => 31 * a + b) } } 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 e340b378e0..c85b3caf8a 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -199,6 +199,7 @@ private[spark] class AppStatusListener( exec.totalCores = event.executorInfo.totalCores exec.maxTasks = event.executorInfo.totalCores / coresPerTask exec.executorLogs = event.executorInfo.logUrlMap + exec.resources = event.executorInfo.resourcesInfo exec.attributes = event.executorInfo.attributes liveUpdate(exec, System.nanoTime()) } 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 c5a233f14a..aa4a21c1bb 100644 --- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala +++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala @@ -27,6 +27,7 @@ import com.google.common.collect.Interners import org.apache.spark.JobExecutionStatus import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} +import org.apache.spark.resource.ResourceInformation import org.apache.spark.scheduler.{AccumulableInfo, StageInfo, TaskInfo} import org.apache.spark.status.api.v1 import org.apache.spark.storage.RDDInfo @@ -259,6 +260,7 @@ private class LiveExecutor(val executorId: String, _addTime: Long) extends LiveE var executorLogs = Map[String, String]() var attributes = Map[String, String]() + var resources = Map[String, ResourceInformation]() // Memory metrics. They may not be recorded (e.g. old event logs) so if totalOnHeap is not // initialized, the store will not contain this information. @@ -308,7 +310,8 @@ private class LiveExecutor(val executorId: String, _addTime: Long) extends LiveE memoryMetrics, blacklistedInStages, Some(peakExecutorMetrics).filter(_.isSet), - attributes) + attributes, + resources) new ExecutorSummaryWrapper(info) } } 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 7b3636fdc5..5ec9b36393 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 @@ -30,6 +30,7 @@ import com.fasterxml.jackson.databind.annotation.{JsonDeserialize, JsonSerialize import org.apache.spark.JobExecutionStatus import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.metrics.ExecutorMetricType +import org.apache.spark.resource.ResourceInformation case class ApplicationInfo private[spark]( id: String, @@ -107,7 +108,8 @@ class ExecutorSummary private[spark]( @JsonSerialize(using = classOf[ExecutorMetricsJsonSerializer]) @JsonDeserialize(using = classOf[ExecutorMetricsJsonDeserializer]) val peakMemoryMetrics: Option[ExecutorMetrics], - val attributes: Map[String, String]) + val attributes: Map[String, String], + val resources: Map[String, ResourceInformation]) class MemoryMetrics private[spark]( val usedOnHeapStorageMemory: Long, diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 6b069754cf..73ef80980e 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -33,6 +33,7 @@ import org.apache.spark._ import org.apache.spark.executor._ import org.apache.spark.metrics.ExecutorMetricType import org.apache.spark.rdd.RDDOperationScope +import org.apache.spark.resource.ResourceInformation import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.storage._ @@ -495,7 +496,15 @@ private[spark] object JsonProtocol { ("Host" -> executorInfo.executorHost) ~ ("Total Cores" -> executorInfo.totalCores) ~ ("Log Urls" -> mapToJson(executorInfo.logUrlMap)) ~ - ("Attributes" -> mapToJson(executorInfo.attributes)) + ("Attributes" -> mapToJson(executorInfo.attributes)) ~ + ("Resources" -> resourcesMapToJson(executorInfo.resourcesInfo)) + } + + def resourcesMapToJson(m: Map[String, ResourceInformation]): JValue = { + val jsonFields = m.map { + case (k, v) => JField(k, v.toJson) + } + JObject(jsonFields.toList) } def blockUpdatedInfoToJson(blockUpdatedInfo: BlockUpdatedInfo): JValue = { @@ -1086,7 +1095,11 @@ private[spark] object JsonProtocol { case Some(attr) => mapFromJson(attr).toMap case None => Map.empty[String, String] } - new ExecutorInfo(executorHost, totalCores, logUrls, attributes) + val resources = jsonOption(json \ "Resources") match { + case Some(resources) => resourcesMapFromJson(resources).toMap + case None => Map.empty[String, ResourceInformation] + } + new ExecutorInfo(executorHost, totalCores, logUrls, attributes, resources) } def blockUpdatedInfoFromJson(json: JValue): BlockUpdatedInfo = { @@ -1098,6 +1111,14 @@ private[spark] object JsonProtocol { BlockUpdatedInfo(blockManagerId, blockId, storageLevel, memorySize, diskSize) } + def resourcesMapFromJson(json: JValue): Map[String, ResourceInformation] = { + val jsonFields = json.asInstanceOf[JObject].obj + jsonFields.map { case JField(k, v) => + val resourceInfo = ResourceInformation.parseJson(v) + (k, resourceInfo) + }.toMap + } + /** -------------------------------- * * Util JSON deserialization methods | * --------------------------------- */ 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 503f610153..6e6d28b6a5 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" : "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_1553914137147_0018", "name" : "LargeBlocks", "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 503f610153..6e6d28b6a5 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" : "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_1553914137147_0018", "name" : "LargeBlocks", "attempts" : [ { diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json index 8a977a7601..eadf27164c 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json @@ -21,5 +21,6 @@ "addTime" : "2015-02-03T16:43:00.906GMT", "executorLogs" : { }, "blacklistedInStages" : [ ], - "attributes" : { } + "attributes" : { }, + "resources" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json index 5b9174a04d..d322485baa 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json @@ -49,7 +49,8 @@ "MajorGCCount" : 3, "MajorGCTime" : 144 }, - "attributes" : { } + "attributes" : { }, + "resources" : { } }, { "id" : "3", "hostPort" : "test-3.vpc.company.com:37641", @@ -114,7 +115,8 @@ "HTTP_SCHEME" : "http://", "NM_HOST" : "test-3.vpc.company.com", "CONTAINER_ID" : "container_1553914137147_0018_01_000004" - } + }, + "resources" : { } }, { "id" : "2", "hostPort" : "test-4.vpc.company.com:33179", @@ -179,7 +181,8 @@ "HTTP_SCHEME" : "http://", "NM_HOST" : "test-4.vpc.company.com", "CONTAINER_ID" : "container_1553914137147_0018_01_000003" - } + }, + "resources" : { } }, { "id" : "1", "hostPort" : "test-2.vpc.company.com:43764", @@ -244,5 +247,6 @@ "HTTP_SCHEME" : "http://", "NM_HOST" : "test-2.vpc.company.com", "CONTAINER_ID" : "container_1553914137147_0018_01_000002" - } + }, + "resources" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json index c7f0a86211..7c3f77d8c1 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json @@ -27,7 +27,8 @@ "totalOffHeapStorageMemory" : 524288000 }, "blacklistedInStages" : [ ], - "attributes" : { } + "attributes" : { }, + "resources" : { } }, { "id" : "3", "hostPort" : "172.22.0.167:51485", @@ -60,7 +61,8 @@ "totalOffHeapStorageMemory" : 524288000 }, "blacklistedInStages" : [ ], - "attributes" : { } + "attributes" : { }, + "resources" : { } } ,{ "id" : "2", "hostPort" : "172.22.0.167:51487", @@ -93,7 +95,8 @@ "totalOffHeapStorageMemory" : 524288000 }, "blacklistedInStages" : [ ], - "attributes" : { } + "attributes" : { }, + "resources" : { } }, { "id" : "1", "hostPort" : "172.22.0.167:51490", @@ -126,7 +129,8 @@ "totalOffHeapStorageMemory": 524288000 }, "blacklistedInStages" : [ ], - "attributes" : { } + "attributes" : { }, + "resources" : { } }, { "id" : "0", "hostPort" : "172.22.0.167:51491", @@ -159,5 +163,6 @@ "totalOffHeapStorageMemory" : 524288000 }, "blacklistedInStages" : [ ], - "attributes" : { } + "attributes" : { }, + "resources" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json index 02a0ecd7f4..0986e85f16 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json @@ -27,7 +27,8 @@ "totalOffHeapStorageMemory" : 524288000 }, "blacklistedInStages" : [ ], - "attributes" : { } + "attributes" : { }, + "resources" : { } }, { "id" : "3", "hostPort" : "172.22.0.167:51485", @@ -60,7 +61,8 @@ "totalOffHeapStorageMemory" : 524288000 }, "blacklistedInStages" : [ ], - "attributes" : { } + "attributes" : { }, + "resources" : { } }, { "id" : "2", "hostPort" : "172.22.0.167:51487", @@ -93,7 +95,8 @@ "totalOffHeapStorageMemory" : 524288000 }, "blacklistedInStages" : [ ], - "attributes" : { } + "attributes" : { }, + "resources" : { } }, { "id" : "1", "hostPort" : "172.22.0.167:51490", @@ -126,7 +129,8 @@ "totalOffHeapStorageMemory": 524288000 }, "blacklistedInStages" : [ ], - "attributes" : { } + "attributes" : { }, + "resources" : { } }, { "id" : "0", "hostPort" : "172.22.0.167:51491", @@ -159,5 +163,6 @@ "totalOffHeapStorageMemory": 524288000 }, "blacklistedInStages" : [ ], - "attributes" : { } + "attributes" : { }, + "resources" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json index 4d31a387fb..26d665151a 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json @@ -21,7 +21,8 @@ "addTime" : "2016-11-15T23:20:38.836GMT", "executorLogs" : { }, "blacklistedInStages" : [ ], - "attributes" : { } + "attributes" : { }, + "resources" : { } }, { "id" : "3", "hostPort" : "172.22.0.111:64543", @@ -48,7 +49,8 @@ "stderr" : "http://172.22.0.111:64521/logPage/?appId=app-20161115172038-0000&executorId=3&logType=stderr" }, "blacklistedInStages" : [ ], - "attributes" : { } + "attributes" : { }, + "resources" : { } }, { "id" : "2", "hostPort" : "172.22.0.111:64539", @@ -75,7 +77,8 @@ "stderr" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stderr" }, "blacklistedInStages" : [ ], - "attributes" : { } + "attributes" : { }, + "resources" : { } }, { "id" : "1", "hostPort" : "172.22.0.111:64541", @@ -102,7 +105,8 @@ "stderr" : "http://172.22.0.111:64518/logPage/?appId=app-20161115172038-0000&executorId=1&logType=stderr" }, "blacklistedInStages" : [ ], - "attributes" : { } + "attributes" : { }, + "resources" : { } }, { "id" : "0", "hostPort" : "172.22.0.111:64540", @@ -129,5 +133,6 @@ "stderr" : "http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stderr" }, "blacklistedInStages" : [ ], - "attributes" : { } + "attributes" : { }, + "resources" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_resource_information_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_resource_information_expectation.json new file mode 100644 index 0000000000..e69ab3b49d --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/executor_resource_information_expectation.json @@ -0,0 +1,130 @@ +[ { + "id" : "driver", + "hostPort" : "10.28.9.112:37319", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 0, + "maxTasks" : 0, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 0, + "totalTasks" : 0, + "totalDuration" : 0, + "totalGCTime" : 0, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : false, + "maxMemory" : 384093388, + "addTime" : "2019-07-02T21:02:25.595GMT", + "executorLogs" : { }, + "memoryMetrics" : { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 384093388, + "totalOffHeapStorageMemory" : 0 + }, + "blacklistedInStages" : [ ], + "attributes" : { }, + "resources" : { } +}, { + "id" : "2", + "hostPort" : "tomg-test:46005", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 1, + "maxTasks" : 1, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 0, + "totalTasks" : 0, + "totalDuration" : 0, + "totalGCTime" : 0, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : false, + "maxMemory" : 384093388, + "addTime" : "2019-07-02T21:02:29.256GMT", + "executorLogs" : { + "stdout" : "http://tomg-test:8042/node/containerlogs/container_1555004656427_0144_01_000003/tgraves/stdout?start=-4096", + "stderr" : "http://tomg-test:8042/node/containerlogs/container_1555004656427_0144_01_000003/tgraves/stderr?start=-4096" + }, + "memoryMetrics" : { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 384093388, + "totalOffHeapStorageMemory" : 0 + }, + "blacklistedInStages" : [ ], + "attributes" : { + "NM_HTTP_ADDRESS" : "tomg-test:8042", + "USER" : "tgraves", + "LOG_FILES" : "stderr,stdout", + "NM_HTTP_PORT" : "8042", + "CLUSTER_ID" : "", + "NM_PORT" : "43125", + "HTTP_SCHEME" : "http://", + "NM_HOST" : "tomg-test", + "CONTAINER_ID" : "container_1555004656427_0144_01_000003" + }, + "resources" : { + "gpu" : { + "name" : "gpu", + "addresses" : [ "0", "1", "2", "3", "4", "5", "6", "7", "8", "9", "10", "11", "12" ] + } + } +}, { + "id" : "1", + "hostPort" : "tomg-test:44873", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 1, + "maxTasks" : 1, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 0, + "totalTasks" : 0, + "totalDuration" : 0, + "totalGCTime" : 0, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : false, + "maxMemory" : 384093388, + "addTime" : "2019-07-02T21:02:28.551GMT", + "executorLogs" : { + "stdout" : "http://tomg-test:8042/node/containerlogs/container_1555004656427_0144_01_000002/tgraves/stdout?start=-4096", + "stderr" : "http://tomg-test:8042/node/containerlogs/container_1555004656427_0144_01_000002/tgraves/stderr?start=-4096" + }, + "memoryMetrics" : { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 384093388, + "totalOffHeapStorageMemory" : 0 + }, + "blacklistedInStages" : [ ], + "attributes" : { + "NM_HTTP_ADDRESS" : "tomg-test:8042", + "USER" : "tgraves", + "LOG_FILES" : "stderr,stdout", + "NM_HTTP_PORT" : "8042", + "CLUSTER_ID" : "", + "NM_PORT" : "43125", + "HTTP_SCHEME" : "http://", + "NM_HOST" : "tomg-test", + "CONTAINER_ID" : "container_1555004656427_0144_01_000002" + }, + "resources" : { + "gpu" : { + "name" : "gpu", + "addresses" : [ "0", "1", "2", "3", "4", "5", "6", "7", "8", "9", "10", "11", "12" ] + } + } +} ] 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 62e068217c..3102909f81 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" : "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_1553914137147_0018", "name" : "LargeBlocks", "attempts" : [ { @@ -28,19 +43,4 @@ "startTimeEpoch" : 1516300235119, "endTimeEpoch" : 1516300707938 } ] -}, { - "id" : "app-20180109111548-0000", - "name" : "Spark shell", - "attempts" : [ { - "startTime" : "2018-01-09T10:15:42.372GMT", - "endTime" : "2018-01-09T10:24:37.606GMT", - "lastUpdated" : "", - "duration" : 535234, - "sparkUser" : "attilapiros", - "completed" : true, - "appSparkVersion" : "2.3.0-SNAPSHOT", - "lastUpdatedEpoch" : 0, - "startTimeEpoch" : 1515492942372, - "endTimeEpoch" : 1515493477606 - } ] } ] 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 a212d544b3..794f1514a6 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,22 @@ -[ { +[ + { + "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_1553914137147_0018", "name" : "LargeBlocks", "attempts" : [ { 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 023e1ed496..adcdccef48 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" : "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_1553914137147_0018", "name" : "LargeBlocks", "attempts" : [ { diff --git a/core/src/test/resources/spark-events/application_1555004656427_0144 b/core/src/test/resources/spark-events/application_1555004656427_0144 new file mode 100644 index 0000000000..91dae7e09e --- /dev/null +++ b/core/src/test/resources/spark-events/application_1555004656427_0144 @@ -0,0 +1,9 @@ +{"Event":"SparkListenerLogStart","Spark Version":"3.0.0-SNAPSHOT"} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"10.28.9.112","Port":37319},"Maximum Memory":384093388,"Timestamp":1562101345595,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/lib/jvm/java-8-openjdk-amd64/jre","Java Version":"1.8.0_212 (Oracle Corporation)","Scala Version":"version 2.12.8"},"Spark Properties":{"spark.executor.resource.gpu.amount":"1","spark.yarn.dist.files":"file:///home/tgraves/getGpus","spark.driver.host":"10.28.9.112","spark.executor.resource.gpu.discoveryScript":"./getGpus","spark.eventLog.enabled":"true","spark.driver.port":"38895","spark.repl.class.uri":"spark://10.28.9.112:38895/classes","spark.jars":"","spark.repl.class.outputDir":"/tmp/spark-f3ed6acc-96b8-420c-9974-b5be5e562fc6/repl-16ea2b82-f737-4619-8f9c-0dada7cab190","spark.driver.resource.gpu.amount":"1","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.executor.instances":"2","spark.submit.pyFiles":"","spark.ui.showConsoleProgress":"true","spark.driver.resource.gpu.discoveryScript":"/home/tgraves/getGpus","spark.executor.id":"driver","spark.task.resource.gpu.amount":"1","spark.submit.deployMode":"client","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.home":"/home/tgraves/workspace/tgravescs-spark","spark.eventLog.dir":"hdfs:///user/tgraves/eventlogdir","spark.sql.catalogImplementation":"in-memory","spark.driver.appUIAddress":"http://10.28.9.112:4040","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"tomg-test","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://tomg-test:8088/proxy/application_1555004656427_0144","spark.app.id":"application_1555004656427_0144"},"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":"yarn","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","dfs.replication":"1","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.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","mapreduce.reduce.env":"HADOOP_MAPRED_HOME=/home/tgraves/runspace/hadoop/lce/hadoop-3.1.2","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":"hdfs://localhost:9000","yarn.minicluster.use-rpc":"false","yarn.app.mapreduce.am.env":"HADOOP_MAPRED_HOME=/home/tgraves/runspace/hadoop/lce/hadoop-3.1.2","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","yarn.nodemanager.linux-container-executor.group":"tgraves","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","yarn.scheduler.capacity.resource-calculator":"org.apache.hadoop.yarn.util.resource.DominantResourceCalculator","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","mapreduce.map.env":"HADOOP_MAPRED_HOME=/home/tgraves/runspace/hadoop/lce/hadoop-3.1.2","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","yarn.nodemanager.aux-services.spark_shuffle.class":"org.apache.spark.network.yarn.YarnShuffleService","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":"false","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":"true","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.LinuxContainerExecutor","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.aux-services":"mapreduce_shuffle,spark_shuffle","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":"false","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","mapreduce.application.classpath":"$HADOOP_MAPRED_HOME/share/hadoop/mapreduce/*:$HADOOP_MAPRED_HOME/share/hadoop/mapreduce/lib/*","yarn.client.nodemanager-connect.retry-interval-ms":"10000","yarn.timeline-service.state-store-class":"org.apache.hadoop.yarn.server.timeline.recovery.LeveldbTimelineStateStore","yarn.nodemanager.env-whitelist":"JAVA_HOME,HADOOP_COMMON_HOME,HADOOP_HDFS_HOME,HADOOP_CONF_DIR,CLASSPATH_PREPEND_DISTCACHE,HADOOP_YARN_HOME,HADOOP_HOME,PATH,LANG,TZ","yarn.sharedcache.nested-level":"3","yarn.timeline-service.webapp.rest-csrf.methods-to-ignore":"GET,OPTIONS,HEAD","fs.azure.user.agent.prefix":"unknown","yarn.resourcemanager.zk-delegation-token-node.split-index":"*********(redacted)","yarn.nodemanager.numa-awareness.read-topology":"false","yarn.nodemanager.webapp.address":"${yarn.nodemanager.hostname}:8042","rpc.metrics.quantile.enable":"false","yarn.registry.class":"org.apache.hadoop.registry.client.impl.FSRegistryOperationsService","mapreduce.jobhistory.admin.acl":"*","yarn.resourcemanager.system-metrics-publisher.dispatcher.pool-size":"10","yarn.scheduler.queue-placement-rules":"user-group","hadoop.http.authentication.kerberos.keytab":"${user.home}/hadoop.keytab","yarn.resourcemanager.recovery.enabled":"false","yarn.timeline-service.webapp.rest-csrf.enabled":"false"},"System Properties":{"java.io.tmpdir":"/tmp","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"/home/tgraves","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/amd64","user.dir":"/home/tgraves/workspace/tgravescs-spark","java.library.path":"/usr/java/packages/lib/amd64:/usr/lib/x86_64-linux-gnu/jni:/lib/x86_64-linux-gnu:/usr/lib/x86_64-linux-gnu:/usr/lib/jni:/lib:/usr/lib","sun.cpu.isalist":"","sun.desktop":"gnome","os.arch":"amd64","java.vm.version":"25.212-b03","jetty.git.hash":"e1bc35120a6617ee3df052294e433f3a25ce7097","java.endorsed.dirs":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/endorsed","java.runtime.version":"1.8.0_212-8u212-b03-0ubuntu1.18.04.1-b03","java.vm.info":"mixed mode","java.ext.dirs":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"OpenJDK Runtime Environment","file.separator":"/","java.class.version":"52.0","scala.usejavacp":"true","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/resources.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/rt.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/sunrsasign.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jsse.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jce.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/charsets.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/lib/jfr.jar:/usr/lib/jvm/java-8-openjdk-amd64/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"4.15.0-46-generic","sun.os.patch.level":"unknown","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","user.language":"en","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.print.PSPrinterJob","java.awt.graphicsenv":"sun.awt.X11GraphicsEnvironment","awt.toolkit":"sun.awt.X11.XToolkit","os.name":"Linux","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"tgraves","java.vm.name":"OpenJDK 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master yarn --conf spark.executor.resource.gpu.amount=1 --conf spark.driver.resource.gpu.amount=1 --conf spark.eventLog.enabled=true --conf spark.eventLog.dir=hdfs:///user/tgraves/eventlogdir --conf spark.executor.resource.gpu.discoveryScript=./getGpus --conf spark.driver.resource.gpu.discoveryScript=/home/tgraves/getGpus --conf spark.task.resource.gpu.amount=1 --class org.apache.spark.repl.Main --name Spark shell --driver-cores 2 --num-executors 2 --files file:/home/tgraves/getGpus spark-shell","java.home":"/usr/lib/jvm/java-8-openjdk-amd64/jre","java.version":"1.8.0_212","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/stream-2.9.6.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/spark-repl_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/hadoop-yarn-registry-3.2.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/kerb-crypto-1.0.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/spark-tags_2.12-3.0.0-SNAPSHOT-tests.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/kerby-asn1-1.0.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/aircompressor-0.10.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/jersey-container-servlet-2.22.2.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/jersey-server-2.22.2.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/jackson-jaxrs-base-2.9.5.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/guava-14.0.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/zjsonpatch-0.3.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/slf4j-log4j12-1.7.16.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/parquet-jackson-1.10.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/kerb-core-1.0.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/token-provider-1.0.1.jar":"*********(redacted)","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/commons-beanutils-1.9.3.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/metrics-graphite-3.1.5.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/metrics-jvm-3.1.5.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/istack-commons-runtime-3.0.8.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/minlog-1.3.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/dnsjava-2.1.7.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/hadoop-yarn-client-3.2.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/commons-net-3.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/macro-compat_2.12-1.1.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/shapeless_2.12-2.3.2.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/spark-streaming_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/jackson-databind-2.9.9.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/jetty-xml-9.4.18.v20190429.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/json-smart-2.3.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/guice-servlet-4.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/orc-shims-1.5.5.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/scala-xml_2.12-1.2.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/jersey-common-2.22.2.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/parquet-format-2.4.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/spark-catalyst_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/jackson-module-paranamer-2.9.9.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/aopalliance-repackaged-2.4.0-b34.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/jtransforms-2.4.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/kerb-util-1.0.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/commons-codec-1.10.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/netty-all-4.1.30.Final.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/javax.ws.rs-api-2.0.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/spark-launcher_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/hadoop-client-3.2.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/xz-1.5.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/spark-yarn_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/kubernetes-model-4.1.2.jar":"System Classpath","/hadoopconfs/":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/ivy-2.4.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/avro-mapred-1.8.2-hadoop2.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/parquet-column-1.10.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/geronimo-jcache_1.0_spec-1.0-alpha-1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/hk2-api-2.4.0-b34.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/slf4j-api-1.7.16.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/spark-graph-api_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/leveldbjni-all-1.8.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/commons-lang-2.6.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/compress-lzf-1.0.3.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/nimbus-jose-jwt-4.41.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/re2j-1.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/hadoop-mapreduce-client-common-3.2.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/chill-java-0.9.3.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/kubernetes-model-common-4.1.2.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/kerb-client-1.0.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/automaton-1.11-8.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/joda-time-2.9.3.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/hadoop-yarn-api-3.2.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/jersey-media-jaxb-2.22.2.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/conf/":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/commons-lang3-3.8.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/arpack_combined_all-0.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/spark-kvstore_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/core-1.1.2.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/arrow-format-0.12.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/accessors-smart-1.2.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/commons-crypto-1.0.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/okhttp-3.8.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/snakeyaml-1.23.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/breeze_2.12-0.13.2.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/orc-core-1.5.5.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/curator-client-2.13.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/spark-graphx_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/spark-core_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/antlr4-runtime-4.7.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/commons-text-1.6.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/activation-1.1.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/kerb-simplekdc-1.0.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/jsr305-3.0.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/paranamer-2.8.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/commons-math3-3.4.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/jackson-core-2.9.9.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/hadoop-mapreduce-client-jobclient-3.2.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/kerb-common-1.0.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/hadoop-hdfs-client-3.2.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/jackson-module-scala_2.12-2.9.9.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/lz4-java-1.6.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/httpcore-4.4.10.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/kerb-server-1.0.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/commons-cli-1.2.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/parquet-hadoop-1.10.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/curator-recipes-2.13.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/logging-interceptor-3.12.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/kerby-pkix-1.0.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/woodstox-core-5.0.3.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/hadoop-annotations-3.2.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/json4s-ast_2.12-3.6.6.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/univocity-parsers-2.7.3.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/xbean-asm7-shaded-4.14.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/parquet-encoding-1.10.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/spark-sketch_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/jersey-container-servlet-core-2.22.2.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/spark-graph_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/osgi-resource-locator-1.0.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/hk2-locator-2.4.0-b34.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/jackson-dataformat-yaml-2.9.9.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/javax.inject-2.4.0-b34.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/aopalliance-1.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/commons-compiler-3.0.11.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/netty-3.9.9.Final.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/spark-mllib_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/scala-library-2.12.8.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/javax.inject-1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/hadoop-auth-3.2.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/spark-network-common_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/okio-1.13.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/spire-macros_2.12-0.13.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/arrow-memory-0.12.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/spark-sql_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/hppc-0.7.2.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/spire_2.12-0.13.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/arrow-vector-0.12.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/opencsv-2.3.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/commons-configuration2-2.1.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/jackson-annotations-2.9.9.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/spark-mllib-local_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/pyrolite-4.23.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/json4s-jackson_2.12-3.6.6.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/jersey-client-2.22.2.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/ehcache-3.3.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/spark-tags_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/machinist_2.12-0.6.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/hadoop-yarn-server-web-proxy-3.2.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/javassist-3.18.1-GA.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/orc-mapreduce-1.5.5.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/hive-storage-api-2.6.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/hadoop-mapreduce-client-core-3.2.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/log4j-1.2.17.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/htrace-core4-4.1.0-incubating.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/avro-ipc-1.8.2.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/jcl-over-slf4j-1.7.16.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/spark-kubernetes_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/snappy-java-1.1.7.3.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/oro-2.0.8.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/kubernetes-client-4.1.2.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/generex-1.0.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/spark-cypher_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/RoaringBitmap-0.7.45.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/commons-io-2.4.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/commons-daemon-1.0.13.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/avro-1.8.2.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/chill_2.12-0.9.3.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/janino-3.0.11.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/hadoop-yarn-server-common-3.2.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/stax2-api-3.1.4.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/zookeeper-3.4.13.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/breeze-macros_2.12-0.13.2.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/spark-network-shuffle_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/jaxb-api-2.2.11.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/spark-unsafe_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/guice-4.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/scala-compiler-2.12.8.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/jsp-api-2.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/metrics-json-3.1.5.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/kerby-xdr-1.0.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/httpclient-4.5.6.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/shims-0.7.45.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/py4j-0.10.8.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/jackson-module-jaxb-annotations-2.9.9.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/validation-api-1.1.0.Final.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/kerb-identity-1.0.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/scala-parser-combinators_2.12-1.1.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/audience-annotations-0.5.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/jul-to-slf4j-1.7.16.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/jcip-annotations-1.0-1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/objenesis-2.5.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/commons-compress-1.8.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/kerb-admin-1.0.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/okhttp-2.7.5.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/metrics-core-3.1.5.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/kryo-shaded-4.0.2.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/jaxb-runtime-2.3.2.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/json4s-scalap_2.12-3.6.6.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/javax.servlet-api-3.1.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/kerby-config-1.0.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/flatbuffers-java-1.9.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/curator-framework-2.13.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/jetty-webapp-9.4.18.v20190429.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/hadoop-yarn-common-3.2.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/gson-2.2.4.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/scala-reflect-2.12.8.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/hk2-utils-2.4.0-b34.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/hadoop-common-3.2.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/kerby-util-1.0.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/jackson-jaxrs-json-provider-2.9.5.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/javax.annotation-api-1.2.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/jakarta.xml.bind-api-2.3.2.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/parquet-common-1.10.1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/zstd-jni-1.4.0-1.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/jersey-guava-2.22.2.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/commons-collections-3.2.2.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/json4s-core_2.12-3.6.6.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/protobuf-java-2.5.0.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/okapi-shade-0.4.2.jar":"System Classpath","/home/tgraves/workspace/tgravescs-spark/assembly/target/scala-2.12/jars/mssql-jdbc-6.2.1.jre7.jar":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"application_1555004656427_0144","Timestamp":1562101337180,"User":"tgraves"} +{"Event":"SparkListenerExecutorAdded","Timestamp":1562101348551,"Executor ID":"1","Executor Info":{"Host":"tomg-test","Total Cores":1,"Log Urls":{"stdout":"http://tomg-test:8042/node/containerlogs/container_1555004656427_0144_01_000002/tgraves/stdout?start=-4096","stderr":"http://tomg-test:8042/node/containerlogs/container_1555004656427_0144_01_000002/tgraves/stderr?start=-4096"},"Attributes":{"NM_HTTP_ADDRESS":"tomg-test:8042","USER":"tgraves","LOG_FILES":"stderr,stdout","NM_HTTP_PORT":"8042","CLUSTER_ID":"","NM_PORT":"43125","HTTP_SCHEME":"http://","NM_HOST":"tomg-test","CONTAINER_ID":"container_1555004656427_0144_01_000002"},"Resources":{"gpu":{"name":"gpu","addresses":["0","1","2","3","4","5","6","7","8","9","10","11","12"]}}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"tomg-test","Port":44873},"Maximum Memory":384093388,"Timestamp":1562101348609,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"Event":"SparkListenerExecutorAdded","Timestamp":1562101349256,"Executor ID":"2","Executor Info":{"Host":"tomg-test","Total Cores":1,"Log Urls":{"stdout":"http://tomg-test:8042/node/containerlogs/container_1555004656427_0144_01_000003/tgraves/stdout?start=-4096","stderr":"http://tomg-test:8042/node/containerlogs/container_1555004656427_0144_01_000003/tgraves/stderr?start=-4096"},"Attributes":{"NM_HTTP_ADDRESS":"tomg-test:8042","USER":"tgraves","LOG_FILES":"stderr,stdout","NM_HTTP_PORT":"8042","CLUSTER_ID":"","NM_PORT":"43125","HTTP_SCHEME":"http://","NM_HOST":"tomg-test","CONTAINER_ID":"container_1555004656427_0144_01_000003"},"Resources":{"gpu":{"name":"gpu","addresses":["0","1","2","3","4","5","6","7","8","9","10","11","12"]}}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"2","Host":"tomg-test","Port":46005},"Maximum Memory":384093388,"Timestamp":1562101349311,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"Event":"SparkListenerApplicationEnd","Timestamp":1562101355974} diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index c1402bd291..786f55c96a 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -783,8 +783,9 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu .set(DRIVER_RESOURCES_FILE, resourcesFile) .setMaster("local-cluster[1, 1, 1024]") .setAppName("test-cluster") - conf.set(DRIVER_GPU_ID.amountConf, "3") - conf.set(DRIVER_GPU_ID.discoveryScriptConf, scriptPath) + .set(DRIVER_GPU_ID.amountConf, "3") + .set(DRIVER_GPU_ID.discoveryScriptConf, scriptPath) + .set(SPARK_RESOURCES_DIR, dir.getName()) sc = new SparkContext(conf) @@ -855,10 +856,11 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu val conf = new SparkConf() .setMaster("local-cluster[3, 3, 1024]") .setAppName("test-cluster") - conf.set(WORKER_GPU_ID.amountConf, "3") - conf.set(WORKER_GPU_ID.discoveryScriptConf, discoveryScript) - conf.set(TASK_GPU_ID.amountConf, "3") - conf.set(EXECUTOR_GPU_ID.amountConf, "3") + .set(WORKER_GPU_ID.amountConf, "3") + .set(WORKER_GPU_ID.discoveryScriptConf, discoveryScript) + .set(TASK_GPU_ID.amountConf, "3") + .set(EXECUTOR_GPU_ID.amountConf, "3") + .set(SPARK_RESOURCES_DIR, dir.getName()) sc = new SparkContext(conf) 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 3f8c90e889..dbc1938ed4 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 @@ -170,6 +170,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers "executor node blacklisting" -> "applications/app-20161116163331-0000/executors", "executor node blacklisting unblacklisting" -> "applications/app-20161115172038-0000/executors", "executor memory usage" -> "applications/app-20161116163331-0000/executors", + "executor resource information" -> "applications/application_1555004656427_0144/executors", "app environment" -> "applications/app-20161116163331-0000/environment", diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index bbf64be343..e781c5f71f 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -32,6 +32,8 @@ import org.apache.spark._ import org.apache.spark.executor._ import org.apache.spark.metrics.ExecutorMetricType import org.apache.spark.rdd.RDDOperationScope +import org.apache.spark.resource.ResourceInformation +import org.apache.spark.resource.ResourceUtils import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.shuffle.MetadataFetchFailedException @@ -83,13 +85,15 @@ class JsonProtocolSuite extends SparkFunSuite { val unpersistRdd = SparkListenerUnpersistRDD(12345) val logUrlMap = Map("stderr" -> "mystderr", "stdout" -> "mystdout").toMap val attributes = Map("ContainerId" -> "ct1", "User" -> "spark").toMap + val resources = Map(ResourceUtils.GPU -> + new ResourceInformation(ResourceUtils.GPU, Array("0", "1"))) val applicationStart = SparkListenerApplicationStart("The winner of all", Some("appId"), 42L, "Garfield", Some("appAttempt")) val applicationStartWithLogs = SparkListenerApplicationStart("The winner of all", Some("appId"), 42L, "Garfield", Some("appAttempt"), Some(logUrlMap)) val applicationEnd = SparkListenerApplicationEnd(42L) val executorAdded = SparkListenerExecutorAdded(executorAddedTime, "exec1", - new ExecutorInfo("Hostee.awesome.com", 11, logUrlMap, attributes)) + new ExecutorInfo("Hostee.awesome.com", 11, logUrlMap, attributes, resources.toMap)) val executorRemoved = SparkListenerExecutorRemoved(executorRemovedTime, "exec2", "test reason") val executorBlacklisted = SparkListenerExecutorBlacklisted(executorBlacklistedTime, "exec1", 22) val executorUnblacklisted = @@ -1947,6 +1951,12 @@ private[spark] object JsonProtocolSuite extends Assertions { | "Attributes" : { | "ContainerId" : "ct1", | "User" : "spark" + | }, + | "Resources" : { + | "gpu" : { + | "name" : "gpu", + | "addresses" : [ "0", "1" ] + | } | } | } |} diff --git a/dev/.rat-excludes b/dev/.rat-excludes index 6d8e57825f..e12dc994b0 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -83,6 +83,7 @@ app-20161115172038-0000 app-20161116163331-0000 application_1516285256255_0012 application_1553914137147_0018 +application_1555004656427_0144 stat local-1422981759269 local-1422981780767